diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidVoterKeyException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidVoterKeyException.java new file mode 100644 index 00000000000..c417713e349 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidVoterKeyException.java @@ -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); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index cba3e439acf..ded58495e6d 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -77,6 +77,7 @@ import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.InvalidTxnTimeoutException; import org.apache.kafka.common.errors.InvalidUpdateVersionException; +import org.apache.kafka.common.errors.InvalidVoterKeyException; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.errors.ListenerNotFoundException; @@ -403,7 +404,8 @@ public enum Errors { INVALID_RECORD_STATE(121, "The record state is invalid. The acknowledgement of delivery could not be completed.", InvalidRecordStateException::new), SHARE_SESSION_NOT_FOUND(122, "The share session was not found.", ShareSessionNotFoundException::new), INVALID_SHARE_SESSION_EPOCH(123, "The share session epoch is invalid.", InvalidShareSessionEpochException::new), - FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new); + FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new), + INVALID_VOTER_KEY(125, "The voter key doesn't match the receiving replica's key.", InvalidVoterKeyException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java index 0794fb46095..58998933df8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java @@ -68,16 +68,12 @@ public class BeginQuorumEpochRequest extends AbstractRequest { return new BeginQuorumEpochRequest(new BeginQuorumEpochRequestData(new ByteBufferAccessor(buffer), version), version); } - public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition, - int leaderEpoch, - int leaderId) { - return singletonRequest(topicPartition, null, leaderEpoch, leaderId); - } - - public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition, - String clusterId, - int leaderEpoch, - int leaderId) { + public static BeginQuorumEpochRequestData singletonRequest( + TopicPartition topicPartition, + String clusterId, + int leaderEpoch, + int leaderId + ) { return new BeginQuorumEpochRequestData() .setClusterId(clusterId) .setTopics(Collections.singletonList( @@ -90,5 +86,4 @@ public class BeginQuorumEpochRequest extends AbstractRequest { .setLeaderId(leaderId)))) ); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java index 5ae975acd8a..b96728351af 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java @@ -17,14 +17,12 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -49,27 +47,6 @@ public class BeginQuorumEpochResponse extends AbstractResponse { this.data = data; } - public static BeginQuorumEpochResponseData singletonResponse( - Errors topLevelError, - TopicPartition topicPartition, - Errors partitionLevelError, - int leaderEpoch, - int leaderId - ) { - return new BeginQuorumEpochResponseData() - .setErrorCode(topLevelError.code()) - .setTopics(Collections.singletonList( - new BeginQuorumEpochResponseData.TopicData() - .setTopicName(topicPartition.topic()) - .setPartitions(Collections.singletonList( - new BeginQuorumEpochResponseData.PartitionData() - .setErrorCode(partitionLevelError.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(leaderEpoch) - ))) - ); - } - @Override public Map errorCounts() { Map errors = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java index 136bc54d50b..659c4f7ef22 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -26,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; public class EndQuorumEpochRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { @@ -95,4 +97,18 @@ public class EndQuorumEpochRequest extends AbstractRequest { ); } + public static List 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(); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java index b3a236adc69..37ca81deca6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java @@ -17,14 +17,12 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -78,27 +76,6 @@ public class EndQuorumEpochResponse extends AbstractResponse { // Not supported by the response schema } - public static EndQuorumEpochResponseData singletonResponse( - Errors topLevelError, - TopicPartition topicPartition, - Errors partitionLevelError, - int leaderEpoch, - int leaderId - ) { - return new EndQuorumEpochResponseData() - .setErrorCode(topLevelError.code()) - .setTopics(Collections.singletonList( - new EndQuorumEpochResponseData.TopicData() - .setTopicName(topicPartition.topic()) - .setPartitions(Collections.singletonList( - new EndQuorumEpochResponseData.PartitionData() - .setErrorCode(partitionLevelError.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(leaderEpoch) - ))) - ); - } - public static EndQuorumEpochResponse parse(ByteBuffer buffer, short version) { return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java index e57e7be7e4a..ca79d396dc6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java @@ -23,11 +23,9 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; -import java.util.function.UnaryOperator; public final class FetchSnapshotResponse extends AbstractResponse { private final FetchSnapshotResponseData data; @@ -81,33 +79,6 @@ public final class FetchSnapshotResponse extends AbstractResponse { return new FetchSnapshotResponseData().setErrorCode(error.code()); } - /** - * Creates a FetchSnapshotResponseData with a single PartitionSnapshot for the topic partition. - * - * The partition index will already be populated when calling operator. - * - * @param topicPartition the topic partition to include - * @param operator unary operator responsible for populating all of the appropriate fields - * @return the created fetch snapshot response data - */ - public static FetchSnapshotResponseData singleton( - TopicPartition topicPartition, - UnaryOperator 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. * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java index 8fba2f085d5..531c33b5f83 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java @@ -69,19 +69,6 @@ public class VoteRequest extends AbstractRequest { return new VoteRequest(new VoteRequestData(new ByteBufferAccessor(buffer), version), version); } - public static VoteRequestData singletonRequest(TopicPartition topicPartition, - int candidateEpoch, - int candidateId, - int lastEpoch, - long lastEpochEndOffset) { - return singletonRequest(topicPartition, - null, - candidateEpoch, - candidateId, - lastEpoch, - lastEpochEndOffset); - } - public static VoteRequestData singletonRequest(TopicPartition topicPartition, String clusterId, int candidateEpoch, @@ -102,5 +89,4 @@ public class VoteRequest extends AbstractRequest { .setLastOffset(lastEpochEndOffset)) ))); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java index f79c6eeb0de..c9a64743ccf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java @@ -17,14 +17,12 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -49,25 +47,6 @@ public class VoteResponse extends AbstractResponse { this.data = data; } - public static VoteResponseData singletonResponse(Errors topLevelError, - TopicPartition topicPartition, - Errors partitionLevelError, - int leaderEpoch, - int leaderId, - boolean voteGranted) { - return new VoteResponseData() - .setErrorCode(topLevelError.code()) - .setTopics(Collections.singletonList( - new VoteResponseData.TopicData() - .setTopicName(topicPartition.topic()) - .setPartitions(Collections.singletonList( - new VoteResponseData.PartitionData() - .setErrorCode(partitionLevelError.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(leaderEpoch) - .setVoteGranted(voteGranted))))); - } - @Override public Map errorCounts() { Map errors = new HashMap<>(); diff --git a/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json b/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json index d9d6d92c882..1e956a1fc94 100644 --- a/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json +++ b/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json @@ -18,24 +18,38 @@ "type": "request", "listeners": ["controller"], "name": "BeginQuorumEpochRequest", - "validVersions": "0", - "flexibleVersions": "none", + // Version 1 adds flexible versions, voter key and leader endpoints (KIP-853) + "validVersions": "0-1", + "flexibleVersions": "1+", "fields": [ { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null"}, + { "name": "VoterId", "type": "int32", "versions": "1+", "entityType": "brokerId", "ignorable": true, + "about": "The voter ID of the receiving replica" }, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionData", - "versions": "0+", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the newly elected leader"}, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The epoch of the newly elected leader"} - ]} - ]} + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name" }, + { "name": "Partitions", "type": "[]PartitionData", + "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index" }, + { "name": "VoterDirectoryId", "type": "uuid", "versions": "1+", "ignorable": true, + "about": "The directory id of the receiving replica" }, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the newly elected leader"}, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The epoch of the newly elected leader"} + ] + } + ] + }, + { "name": "LeaderEndpoints", "type": "[]LeaderEndpoint", "versions": "1+", "ignorable": true, + "about": "Endpoints for the leader", "fields": [ + { "name": "Name", "type": "string", "versions": "1+", "mapKey": true, "about": "The name of the endpoint" }, + { "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" }, + { "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" } + ] + } ] } diff --git a/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json b/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json index 4b7d7f5a958..b8aeba56a45 100644 --- a/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json +++ b/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json @@ -17,25 +17,36 @@ "apiKey": 53, "type": "response", "name": "BeginQuorumEpochResponse", - "validVersions": "0", - "flexibleVersions": "none", + // Version 1 adds flexible versions and leader endpoint (KIP-853) + "validVersions": "0-1", + "flexibleVersions": "1+", "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top level error code."}, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionData", - "versions": "0+", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+"}, - { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the current leader or -1 if the leader is unknown."}, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The latest known leader epoch"} - ]} - ]} + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]PartitionData", + "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+"}, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the current leader or -1 if the leader is unknown."}, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch"} + ] + } + ] + }, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0, + "about": "Endpoints for all leaders enumerated in PartitionData", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "1+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node" }, + { "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" }, + { "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" } + ] + } ] } diff --git a/clients/src/main/resources/common/message/EndQuorumEpochRequest.json b/clients/src/main/resources/common/message/EndQuorumEpochRequest.json index a6e4076412f..0c9c56f3f32 100644 --- a/clients/src/main/resources/common/message/EndQuorumEpochRequest.json +++ b/clients/src/main/resources/common/message/EndQuorumEpochRequest.json @@ -18,26 +18,43 @@ "type": "request", "listeners": ["controller"], "name": "EndQuorumEpochRequest", - "validVersions": "0", - "flexibleVersions": "none", + // Version 1 adds flexible versions, replaces preferred successors with preferred candidates + // and adds leader endpoints (KIP-853) + "validVersions": "0-1", + "flexibleVersions": "1+", "fields": [ { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null"}, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionData", - "versions": "0+", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The current leader ID that is resigning"}, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The current epoch"}, - { "name": "PreferredSuccessors", "type": "[]int32", "versions": "0+", - "about": "A sorted list of preferred successors to start the election"} - ]} - ]} + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]PartitionData", + "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The current leader ID that is resigning"}, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The current epoch"}, + { "name": "PreferredSuccessors", "type": "[]int32", "versions": "0", "ignorable": true, + "about": "A sorted list of preferred successors to start the election" }, + { "name": "PreferredCandidates", "type": "[]ReplicaInfo", "versions": "1+", "ignorable": true, + "about": "A sorted list of preferred candidates to start the election", "fields": [ + { "name": "CandidateId", "type": "int32", "versions": "1+", "entityType": "brokerId" }, + { "name": "CandidateDirectoryId", "type": "uuid", "versions": "1+" } + ] + } + ] + } + ] + }, + { "name": "LeaderEndpoints", "type": "[]LeaderEndpoint", "versions": "1+", "ignorable": true, + "about": "Endpoints for the leader", "fields": [ + { "name": "Name", "type": "string", "versions": "1+", "mapKey": true, "about": "The name of the endpoint" }, + { "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" }, + { "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" } + ] + } ] } diff --git a/clients/src/main/resources/common/message/EndQuorumEpochResponse.json b/clients/src/main/resources/common/message/EndQuorumEpochResponse.json index cd232470458..95aa442017f 100644 --- a/clients/src/main/resources/common/message/EndQuorumEpochResponse.json +++ b/clients/src/main/resources/common/message/EndQuorumEpochResponse.json @@ -17,25 +17,36 @@ "apiKey": 54, "type": "response", "name": "EndQuorumEpochResponse", - "validVersions": "0", - "flexibleVersions": "none", + // Version 1 adds flexible versions and leader endpoint (KIP-853) + "validVersions": "0-1", + "flexibleVersions": "1+", "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top level error code."}, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionData", - "versions": "0+", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+"}, - { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the current leader or -1 if the leader is unknown."}, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The latest known leader epoch"} - ]} - ]} + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]PartitionData", + "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+"}, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the current leader or -1 if the leader is unknown."}, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch"} + ] + } + ] + }, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0, + "about": "Endpoints for all leaders enumerated in PartitionData", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "1+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node" }, + { "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" }, + { "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" } + ] + } ] } diff --git a/clients/src/main/resources/common/message/FetchRequest.json b/clients/src/main/resources/common/message/FetchRequest.json index 235357d004e..228e9e4ff28 100644 --- a/clients/src/main/resources/common/message/FetchRequest.json +++ b/clients/src/main/resources/common/message/FetchRequest.json @@ -55,7 +55,9 @@ // deprecate the old ReplicaId field and set its default value to -1. (KIP-903) // // Version 16 is the same as version 15 (KIP-951). - "validVersions": "0-16", + // + // Version 17 adds directory id support from KIP-853 + "validVersions": "0-17", "deprecatedVersions": "0-3", "flexibleVersions": "12+", "fields": [ @@ -100,7 +102,9 @@ { "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, "about": "The earliest available offset of the follower replica. The field is only used when the request is sent by the follower."}, { "name": "PartitionMaxBytes", "type": "int32", "versions": "0+", - "about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." } + "about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." }, + { "name": "ReplicaDirectoryId", "type": "uuid", "versions": "17+", "taggedVersions": "17+", "tag": 0, + "about": "The directory id of the follower fetching" } ]} ]}, { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false, diff --git a/clients/src/main/resources/common/message/FetchResponse.json b/clients/src/main/resources/common/message/FetchResponse.json index e5f49ba6fde..605c7c3ff62 100644 --- a/clients/src/main/resources/common/message/FetchResponse.json +++ b/clients/src/main/resources/common/message/FetchResponse.json @@ -20,7 +20,7 @@ // // Version 1 adds throttle time. // - // Version 2 and 3 are the same as version 1. + // Version 2 and 3 are the same as version 1. // // Version 4 adds features for transactional consumption. // @@ -47,7 +47,9 @@ // Version 15 is the same as version 14 (KIP-903). // // Version 16 adds the 'NodeEndpoints' field (KIP-951). - "validVersions": "0-16", + // + // Version 17 no changes to the response (KIP-853). + "validVersions": "0-17", "flexibleVersions": "12+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/FetchSnapshotRequest.json b/clients/src/main/resources/common/message/FetchSnapshotRequest.json index 358ef2e3227..9d50722d935 100644 --- a/clients/src/main/resources/common/message/FetchSnapshotRequest.json +++ b/clients/src/main/resources/common/message/FetchSnapshotRequest.json @@ -18,7 +18,8 @@ "type": "request", "listeners": ["controller"], "name": "FetchSnapshotRequest", - "validVersions": "0", + // Version 1 adds replica directory id (KIP-853) + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "taggedVersions": "0+", "tag": 0, @@ -29,23 +30,27 @@ "about": "The maximum bytes to fetch from all of the snapshots" }, { "name": "Topics", "type": "[]TopicSnapshot", "versions": "0+", "about": "The topics to fetch", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The name of the topic to fetch" }, - { "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+", - "about": "The partitions to fetch", "fields": [ - { "name": "Partition", "type": "int32", "versions": "0+", - "about": "The partition index" }, - { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+", - "about": "The current leader epoch of the partition, -1 for unknown leader epoch" }, - { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+", - "about": "The snapshot endOffset and epoch to fetch", - "fields": [ - { "name": "EndOffset", "type": "int64", "versions": "0+" }, - { "name": "Epoch", "type": "int32", "versions": "0+" } - ]}, - { "name": "Position", "type": "int64", "versions": "0+", - "about": "The byte position within the snapshot to start fetching from" } - ]} - ]} + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The name of the topic to fetch" }, + { "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+", + "about": "The partitions to fetch", "fields": [ + { "name": "Partition", "type": "int32", "versions": "0+", + "about": "The partition index" }, + { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+", + "about": "The current leader epoch of the partition, -1 for unknown leader epoch" }, + { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+", + "about": "The snapshot endOffset and epoch to fetch", "fields": [ + { "name": "EndOffset", "type": "int64", "versions": "0+" }, + { "name": "Epoch", "type": "int32", "versions": "0+" } + ] + }, + { "name": "Position", "type": "int64", "versions": "0+", + "about": "The byte position within the snapshot to start fetching from" }, + { "name": "ReplicaDirectoryId", "type": "uuid", "versions": "1+", "taggedVersions": "1+", "tag": 0, + "about": "The directory id of the follower fetching" } + ] + } + ] + } ] } diff --git a/clients/src/main/resources/common/message/FetchSnapshotResponse.json b/clients/src/main/resources/common/message/FetchSnapshotResponse.json index 887a5e44012..e5d391ae02e 100644 --- a/clients/src/main/resources/common/message/FetchSnapshotResponse.json +++ b/clients/src/main/resources/common/message/FetchSnapshotResponse.json @@ -17,7 +17,8 @@ "apiKey": 59, "type": "response", "name": "FetchSnapshotResponse", - "validVersions": "0", + "validVersions": "0-1", + // Version 1 adds leader endpoint (KIP-853) "flexibleVersions": "0+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, @@ -26,34 +27,45 @@ "about": "The top level response error code." }, { "name": "Topics", "type": "[]TopicSnapshot", "versions": "0+", "about": "The topics to fetch.", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The name of the topic to fetch." }, - { "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+", - "about": "The partitions to fetch.", "fields": [ - { "name": "Index", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", - "about": "The error code, or 0 if there was no fetch error." }, - { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+", - "about": "The snapshot endOffset and epoch fetched", - "fields": [ - { "name": "EndOffset", "type": "int64", "versions": "0+" }, - { "name": "Epoch", "type": "int32", "versions": "0+" } - ]}, - { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", - "versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [ - { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the current leader or -1 if the leader is unknown."}, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The latest known leader epoch"} - ]}, - { "name": "Size", "type": "int64", "versions": "0+", - "about": "The total size of the snapshot." }, - { "name": "Position", "type": "int64", "versions": "0+", - "about": "The starting byte position within the snapshot included in the Bytes field." }, - { "name": "UnalignedRecords", "type": "records", "versions": "0+", - "about": "Snapshot data in records format which may not be aligned on an offset boundary" } - ]} - ]} + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The name of the topic to fetch." }, + { "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+", + "about": "The partitions to fetch.", "fields": [ + { "name": "Index", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no fetch error." }, + { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+", + "about": "The snapshot endOffset and epoch fetched", "fields": [ + { "name": "EndOffset", "type": "int64", "versions": "0+" }, + { "name": "Epoch", "type": "int32", "versions": "0+" } + ] + }, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", + "versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the current leader or -1 if the leader is unknown."}, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch"} + ] + }, + { "name": "Size", "type": "int64", "versions": "0+", + "about": "The total size of the snapshot." }, + { "name": "Position", "type": "int64", "versions": "0+", + "about": "The starting byte position within the snapshot included in the Bytes field." }, + { "name": "UnalignedRecords", "type": "records", "versions": "0+", + "about": "Snapshot data in records format which may not be aligned on an offset boundary" } + ] + } + ] + }, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0, + "about": "Endpoints for all current-leaders enumerated in PartitionSnapshot", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "1+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node" }, + { "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" }, + { "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" } + ] + } ] } diff --git a/clients/src/main/resources/common/message/VoteRequest.json b/clients/src/main/resources/common/message/VoteRequest.json index 35583a790bc..b010765cd90 100644 --- a/clients/src/main/resources/common/message/VoteRequest.json +++ b/clients/src/main/resources/common/message/VoteRequest.json @@ -18,30 +18,37 @@ "type": "request", "listeners": ["controller"], "name": "VoteRequest", - "validVersions": "0", + // Version 1 adds voter key and candidate directory id (KIP-853) + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null"}, + { "name": "VoterId", "type": "int32", "versions": "1+", "ignorable": true, "default": "-1", "entityType": "brokerId", + "about": "The replica id of the voter receiving the request" }, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionData", - "versions": "0+", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "CandidateEpoch", "type": "int32", "versions": "0+", - "about": "The bumped epoch of the candidate sending the request"}, - { "name": "CandidateId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the voter sending the request"}, - { "name": "LastOffsetEpoch", "type": "int32", "versions": "0+", - "about": "The epoch of the last record written to the metadata log"}, - { "name": "LastOffset", "type": "int64", "versions": "0+", - "about": "The offset of the last record written to the metadata log"} + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]PartitionData", + "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "CandidateEpoch", "type": "int32", "versions": "0+", + "about": "The bumped epoch of the candidate sending the request"}, + { "name": "CandidateId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The replica id of the voter sending the request"}, + { "name": "CandidateDirectoryId", "type": "uuid", "versions": "1+", "ignorable": true, + "about": "The directory id of the voter sending the request" }, + { "name": "VoterDirectoryId", "type": "uuid", "versions": "1+", "ignorable": true, + "about": "The ID of the voter sending the request"}, + { "name": "LastOffsetEpoch", "type": "int32", "versions": "0+", + "about": "The epoch of the last record written to the metadata log"}, + { "name": "LastOffset", "type": "int64", "versions": "0+", + "about": "The offset of the last record written to the metadata log"} + ] + } ] - } - ] } ] } diff --git a/clients/src/main/resources/common/message/VoteResponse.json b/clients/src/main/resources/common/message/VoteResponse.json index b92d0070c1d..cb59ed89578 100644 --- a/clients/src/main/resources/common/message/VoteResponse.json +++ b/clients/src/main/resources/common/message/VoteResponse.json @@ -17,29 +17,38 @@ "apiKey": 52, "type": "response", "name": "VoteResponse", - "validVersions": "0", + // Version 1 adds leader endpoint (KIP-853) + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top level error code."}, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionData", - "versions": "0+", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", - "about": "The partition index." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+"}, - { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the current leader or -1 if the leader is unknown."}, - { "name": "LeaderEpoch", "type": "int32", "versions": "0+", - "about": "The latest known leader epoch"}, - { "name": "VoteGranted", "type": "bool", "versions": "0+", - "about": "True if the vote was granted and false otherwise"} + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]PartitionData", + "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+"}, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the current leader or -1 if the leader is unknown."}, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch"}, + { "name": "VoteGranted", "type": "bool", "versions": "0+", + "about": "True if the vote was granted and false otherwise"} + ] + } + ] + }, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0, + "about": "Endpoints for all current-leaders enumerated in PartitionData", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "1+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node"}, + { "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" }, + { "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" } ] - } - ] } ] } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 18dd9575fa8..65f19be5356 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -463,7 +463,7 @@ class RequestChannel(val queueSize: Int, } } - /** Get the next request or block until specified time has elapsed + /** Get the next request or block until specified time has elapsed * Check the callback queue and execute first if present since these * requests have already waited in line. */ def receiveRequest(timeout: Long): RequestChannel.BaseRequest = { diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 65ef855640c..c57a95b1f80 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -39,11 +39,12 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} import org.apache.kafka.common.protocol.ApiMessage +import org.apache.kafka.common.requests.RequestContext import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time, Utils} -import org.apache.kafka.raft.{FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog} +import org.apache.kafka.raft.{Endpoints, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.util.KafkaScheduler @@ -122,6 +123,7 @@ object KafkaRaftManager { trait RaftManager[T] { def handleRequest( + context: RequestContext, header: RequestHeader, request: ApiMessage, createdTimeMs: Long @@ -152,6 +154,7 @@ class KafkaRaftManager[T]( threadNamePrefixOpt: Option[String], val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], bootstrapServers: JCollection[InetSocketAddress], + controllerListeners: Endpoints, fatalFaultHandler: FaultHandler ) extends RaftManager[T] with Logging { @@ -214,15 +217,16 @@ class KafkaRaftManager[T]( } override def handleRequest( + context: RequestContext, header: RequestHeader, request: ApiMessage, createdTimeMs: Long ): CompletableFuture[ApiMessage] = { - clientDriver.handleRequest(header, request, createdTimeMs) + clientDriver.handleRequest(context, header, request, createdTimeMs) } private def buildRaftClient(): KafkaRaftClient[T] = { - val client = new KafkaRaftClient( + new KafkaRaftClient( OptionalInt.of(config.nodeId), metadataLogDirUuid, recordSerde, @@ -233,9 +237,9 @@ class KafkaRaftManager[T]( logContext, clusterId, bootstrapServers, + controllerListeners, raftConfig ) - client } private def buildNetworkChannel(): KafkaNetworkChannel = { diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index f4c0ba89f81..12f882032e2 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -679,7 +679,7 @@ class ControllerApis( private def handleRaftRequest(request: RequestChannel.Request, buildResponse: ApiMessage => AbstractResponse): CompletableFuture[Unit] = { val requestBody = request.body[AbstractRequest] - val future = raftManager.handleRequest(request.header, requestBody.data, time.milliseconds()) + val future = raftManager.handleRequest(request.context, request.header, requestBody.data, time.milliseconds()) future.handle[Unit] { (responseData, exception) => val response = if (exception != null) { requestBody.getErrorResponse(exception) @@ -926,7 +926,7 @@ class ControllerApis( CreateDelegationTokenResponse.prepareResponse(request.context.requestVersion, requestThrottleMs, Errors.DELEGATION_TOKEN_REQUEST_NOT_ALLOWED, owner, requester)) CompletableFuture.completedFuture[Unit](()) - } else if (!owner.equals(requester) && + } else if (!owner.equals(requester) && !authHelper.authorize(request.context, CREATE_TOKENS, USER, owner.toString)) { // Requester is always allowed to create token for self requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 7c1f694b080..d362928583d 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -187,9 +187,10 @@ class ControllerServer( credentialProvider, apiVersionManager) - val listenerInfo = ListenerInfo.create(config.controllerListeners.map(_.toJava).asJava). - withWildcardHostnamesResolved(). - withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name))) + val listenerInfo = ListenerInfo + .create(config.effectiveAdvertisedControllerListeners.map(_.toJava).asJava) + .withWildcardHostnamesResolved() + .withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name))) socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port()) val endpointReadyFutures = { @@ -203,7 +204,7 @@ class ControllerServer( config.earlyStartListeners.map(_.value()).asJava)) } - sharedServer.startForController() + sharedServer.startForController(listenerInfo) createTopicPolicy = Option(config. getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy])) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 7e84514c1e7..b436ec974ae 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -788,13 +788,24 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami } } - // Use advertised listeners if defined, fallback to listeners otherwise + def effectiveAdvertisedControllerListeners: Seq[EndPoint] = { + // Only expose controller listeners + advertisedListeners.filter(l => controllerListenerNames.contains(l.listenerName.value())) + } + def effectiveAdvertisedListeners: Seq[EndPoint] = { + // Only expose broker listeners + advertisedListeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value())) + } + + // Use advertised listeners if defined, fallback to listeners otherwise + private def advertisedListeners: Seq[EndPoint] = { val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) - if (advertisedListenersProp != null) + if (advertisedListenersProp != null) { CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) - else - listeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value())) + } else { + listeners + } } private def getInterBrokerListenerNameAndSecurityProtocol: (ListenerName, SecurityProtocol) = { @@ -927,7 +938,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} is not supported in KRaft mode.") } def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = { - require(!advertisedListenerNames.exists(aln => controllerListenerNames.contains(aln.value())), + require(advertisedListenerNames.forall(aln => !controllerListenerNames.contains(aln.value())), s"The advertised.listeners config must not contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the broker role because Kafka clients that send requests via advertised listeners do not send requests to KRaft controllers -- they only send requests to KRaft brokers.") } def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = { @@ -979,11 +990,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // KRaft controller-only validateNonEmptyQuorumVotersForKRaft() validateControlPlaneListenerEmptyForKRaft() - // advertised listeners must be empty when only the controller is configured - require( - getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) == null, - s"The ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} config must be empty when ${KRaftConfigs.PROCESS_ROLES_CONFIG}=controller" - ) // listeners should only contain listeners also enumerated in the controller listener require( effectiveAdvertisedListeners.isEmpty, diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index ecb757c1a89..42c2914b4f9 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -93,6 +93,8 @@ class KafkaRaftServer( override def startup(): Unit = { Mx4jLoader.maybeLoad() + // Controller component must be started before the broker component so that + // the controller endpoints are passed to the KRaft manager controller.foreach(_.startup()) broker.foreach(_.startup()) AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds()) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 23b7e8ebc3d..be2d81ef179 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -52,6 +52,7 @@ import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationF import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble} import org.apache.kafka.metadata.{BrokerState, MetadataRecordSerde, VersionRange} import org.apache.kafka.raft.QuorumConfig +import org.apache.kafka.raft.Endpoints import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.NodeToControllerChannelManager import org.apache.kafka.server.authorizer.Authorizer @@ -440,6 +441,9 @@ class KafkaServer( threadNamePrefix, CompletableFuture.completedFuture(quorumVoters), QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), + // Endpoint information is only needed for KRaft controllers (voters). ZK brokers + // (observers) can never be KRaft controllers + Endpoints.empty(), fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index ea92dd61f5f..4d5771a0669 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -22,15 +22,18 @@ import kafka.server.Server.MetricsPrefix import kafka.server.metadata.BrokerServerMetrics import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time} import org.apache.kafka.controller.metrics.ControllerMetadataMetrics import org.apache.kafka.image.MetadataProvenance import org.apache.kafka.image.loader.MetadataLoader import org.apache.kafka.image.loader.metrics.MetadataLoaderMetrics -import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator} import org.apache.kafka.image.publisher.metrics.SnapshotEmitterMetrics +import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator} +import org.apache.kafka.metadata.ListenerInfo import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble +import org.apache.kafka.raft.Endpoints import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, ProcessTerminatingFaultHandler} @@ -43,7 +46,7 @@ import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.{Collection => JCollection} import java.util.{Map => JMap} - +import scala.jdk.CollectionConverters._ /** * Creates a fault handler. @@ -128,7 +131,7 @@ class SharedServer( */ def startForBroker(): Unit = synchronized { if (!isUsed()) { - start() + start(Endpoints.empty()) } usedByBroker = true } @@ -136,9 +139,22 @@ class SharedServer( /** * The start function called by the controller. */ - def startForController(): Unit = synchronized { + def startForController(listenerInfo: ListenerInfo): Unit = synchronized { if (!isUsed()) { - start() + val endpoints = Endpoints.fromInetSocketAddresses( + listenerInfo + .listeners() + .asScala + .map { case (listenerName, endpoint) => + ( + ListenerName.normalised(listenerName), + InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port()) + ) + } + .toMap + .asJava + ) + start(endpoints) } usedByController = true } @@ -237,7 +253,7 @@ class SharedServer( // Note: snapshot generation does not need to be disabled for a publishing fault. }) - private def start(): Unit = synchronized { + private def start(listenerEndpoints: Endpoints): Unit = synchronized { if (started) { debug("SharedServer has already been started.") } else { @@ -256,6 +272,7 @@ class SharedServer( if (sharedServerConfig.processRoles.contains(ProcessRole.ControllerRole)) { controllerServerMetrics = new ControllerMetadataMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry())) } + val _raftManager = new KafkaRaftManager[ApiMessageAndVersion]( clusterId, sharedServerConfig, @@ -268,6 +285,7 @@ class SharedServer( Some(s"kafka-${sharedServerConfig.nodeId}-raft"), // No dash expected at the end controllerQuorumVotersFuture, bootstrapServers, + listenerEndpoints, raftManagerFaultHandler ) raftManager = _raftManager diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index a9b471b1622..64119ab7ed4 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -94,6 +94,7 @@ class TestRaftRequestHandler( val requestBody = request.body[AbstractRequest] val future = raftManager.handleRequest( + request.context, request.header, requestBody.data, time.milliseconds() diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 0acae6c5dc3..06812f27953 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -17,6 +17,7 @@ package kafka.tools +import java.net.InetSocketAddress import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit} import joptsimple.{OptionException, OptionSpec} @@ -35,7 +36,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} import org.apache.kafka.raft.errors.NotLeaderException -import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient, QuorumConfig} +import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, RaftClient, QuorumConfig} import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.serialization.RecordSerde @@ -84,6 +85,15 @@ class TestRaftServer( () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) + val endpoints = Endpoints.fromInetSocketAddresses( + config.effectiveAdvertisedControllerListeners + .map { endpoint => + (endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port)) + } + .toMap + .asJava + ) + raftManager = new KafkaRaftManager[Array[Byte]]( Uuid.ZERO_UUID.toString, config, @@ -96,6 +106,7 @@ class TestRaftServer( Some(threadNamePrefix), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), + endpoints, new ProcessTerminatingFaultHandler.Builder().build() ) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index dce053489f0..96ca34e8c29 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -439,8 +439,9 @@ class SocketServerTest { @Test def testDisabledRequestIsRejected(): Unit = { val correlationId = 57 - val header = new RequestHeader(ApiKeys.VOTE, 0, "", correlationId) - val request = new VoteRequest.Builder(new VoteRequestData()).build() + val version: Short = 0 + val header = new RequestHeader(ApiKeys.VOTE, version, "", correlationId) + val request = new VoteRequest.Builder(new VoteRequestData()).build(version) val serializedBytes = Utils.toArray(request.serializeWithHeader(header)) val socket = connect() diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index da9d29304e5..13a75007417 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -16,6 +16,7 @@ */ package kafka.raft +import java.net.InetSocketAddress import java.nio.channels.FileChannel import java.nio.channels.OverlappingFileLockException import java.nio.file.{Files, Path, StandardOpenOption} @@ -23,25 +24,26 @@ import java.util.Properties import java.util.concurrent.CompletableFuture import kafka.log.LogManager import kafka.server.KafkaConfig -import kafka.utils.TestUtils import kafka.tools.TestRaftServer.ByteArraySerde +import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.network.SocketServerConfigs +import org.apache.kafka.raft.Endpoints import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.ProcessRole +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.fault.FaultHandler import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource -import org.apache.kafka.server.fault.FaultHandler import org.mockito.Mockito._ import scala.util.Using - +import scala.jdk.CollectionConverters._ class RaftManagerTest { private def createZkBrokerConfig( @@ -107,6 +109,15 @@ class RaftManagerTest { ): KafkaRaftManager[Array[Byte]] = { val topicId = new Uuid(0L, 2L) + val endpoints = Endpoints.fromInetSocketAddresses( + config.effectiveAdvertisedControllerListeners + .map { endpoint => + (endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port)) + } + .toMap + .asJava + ) + new KafkaRaftManager[Array[Byte]]( Uuid.randomUuid.toString, config, @@ -119,6 +130,7 @@ class RaftManagerTest { Option.empty, CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), + endpoints, mock(classOf[FaultHandler]) ) } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index a47ec6f7de4..997a711125a 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -219,6 +219,7 @@ class ControllerApisTest { def testFetchSentToKRaft(): Unit = { when( raftManager.handleRequest( + any(classOf[RequestContext]), any(classOf[RequestHeader]), any(classOf[ApiMessage]), any(classOf[Long]) @@ -231,6 +232,7 @@ class ControllerApisTest { controllerApis.handleFetch(buildRequest(new FetchRequest(new FetchRequestData(), 12))) verify(raftManager).handleRequest( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any() @@ -245,6 +247,7 @@ class ControllerApisTest { when( raftManager.handleRequest( + any(classOf[RequestContext]), any(classOf[RequestHeader]), any(classOf[ApiMessage]), any(classOf[Long]) @@ -262,6 +265,7 @@ class ControllerApisTest { verify(raftManager).handleRequest( + ArgumentMatchers.eq(request.context), ArgumentMatchers.eq(request.header), ArgumentMatchers.eq(fetchRequestData), ArgumentMatchers.eq(initialTimeMs) @@ -285,6 +289,7 @@ class ControllerApisTest { def testFetchSnapshotSentToKRaft(): Unit = { when( raftManager.handleRequest( + any(classOf[RequestContext]), any(classOf[RequestHeader]), any(classOf[ApiMessage]), any(classOf[Long]) @@ -297,6 +302,7 @@ class ControllerApisTest { controllerApis.handleFetchSnapshot(buildRequest(new FetchSnapshotRequest(new FetchSnapshotRequestData(), 0))) verify(raftManager).handleRequest( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any() @@ -1207,7 +1213,7 @@ class ControllerApisTest { val response = new FetchResponseData() val responseFuture = new CompletableFuture[ApiMessage]() val errorResponseFuture = new AtomicReference[AbstractResponse]() - when(raftManager.handleRequest(any(), any(), any())).thenReturn(responseFuture) + when(raftManager.handleRequest(any(), any(), any(), any())).thenReturn(responseFuture) when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ => // Simulate an encoding failure in the initial fetch response throw new UnsupportedVersionException("Something went wrong") diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 8da7a7db7cf..43a7136d4ce 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1294,7 +1294,7 @@ class KafkaConfigTest { } @Test - def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = { + def testControllerListenersCanBeAdvertisedForKRaftCombined(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller") val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" @@ -1304,11 +1304,8 @@ class KafkaConfigTest { props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT,SSL") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092") - assertBadConfigContainingMessage(props, - "The advertised.listeners config must not contain KRaft controller listeners from controller.listener.names when process.roles contains the broker role") - // Valid now - props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "SASL_SSL://C:9094") + // Valid KafkaConfig.fromProps(props) // Also valid if we allow advertised listeners to derive from listeners/controller.listener.names @@ -1317,8 +1314,7 @@ class KafkaConfigTest { } @Test - def testAdvertisedListenersDisallowedForKRaftControllerOnlyRole(): Unit = { - // Test that advertised listeners cannot be set when KRaft and server is controller only. + def testAdvertisedListenersAllowedForKRaftControllerOnlyRole(): Unit = { // Test that listeners must enumerate every controller listener // Test that controller listener must enumerate every listener val correctListeners = "PLAINTEXT://A:9092,SSL://B:9093" @@ -1329,21 +1325,18 @@ class KafkaConfigTest { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners) - props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, incorrectListeners) + props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, correctListeners) props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092") - var expectedExceptionContainsText = "The advertised.listeners config must be empty when process.roles=controller" - assertBadConfigContainingMessage(props, expectedExceptionContainsText) - // Invalid if advertised listeners is explicitly to the set - props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, correctListeners) - assertBadConfigContainingMessage(props, expectedExceptionContainsText) + // Valid + KafkaConfig.fromProps(props) // Invalid if listeners contains names not in controller.listener.names props.remove(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, incorrectListeners) - expectedExceptionContainsText = """The listeners config must only contain KRaft controller listeners from + var expectedExceptionContainsText = """The listeners config must only contain KRaft controller listeners from |controller.listener.names when process.roles=controller""".stripMargin.replaceAll("\n", " ") assertBadConfigContainingMessage(props, expectedExceptionContainsText) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index ce502ea1e22..4e2f3285fb8 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -121,15 +121,20 @@ class ReplicaFetcherThreadTest { @Test def shouldSendLatestRequestVersionsByDefault(): Unit = { - val props = TestUtils.createBrokerConfig(1, "localhost:1234") - val config = KafkaConfig.fromProps(props) - - val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) - when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats])) - - assertEquals(ApiKeys.FETCH.latestVersion, config.interBrokerProtocolVersion.fetchRequestVersion()) - assertEquals(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, config.interBrokerProtocolVersion.offsetForLeaderEpochRequestVersion) - assertEquals(ApiKeys.LIST_OFFSETS.latestVersion, config.interBrokerProtocolVersion.listOffsetRequestVersion) + // Check unstable versions + val testingVersion = MetadataVersion.latestTesting + assertEquals( + ApiKeys.FETCH.latestVersion(true), + testingVersion.fetchRequestVersion + ) + assertEquals( + ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(true), + testingVersion.offsetForLeaderEpochRequestVersion + ) + assertEquals( + ApiKeys.LIST_OFFSETS.latestVersion(true), + testingVersion.listOffsetRequestVersion + ) } @Test diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index d797f50f0d9..33c7df2fc52 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -638,10 +638,10 @@ class RequestQuotaTest extends BaseRequestTest { new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData()) case ApiKeys.VOTE => - new VoteRequest.Builder(VoteRequest.singletonRequest(tp, 1, 2, 0, 10)) + new VoteRequest.Builder(VoteRequest.singletonRequest(tp, null, 1, 2, 0, 10)) case ApiKeys.BEGIN_QUORUM_EPOCH => - new BeginQuorumEpochRequest.Builder(BeginQuorumEpochRequest.singletonRequest(tp, 2, 5)) + new BeginQuorumEpochRequest.Builder(BeginQuorumEpochRequest.singletonRequest(tp, null, 2, 5)) case ApiKeys.END_QUORUM_EPOCH => new EndQuorumEpochRequest.Builder(EndQuorumEpochRequest.singletonRequest( diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 6e4b8b7248f..575d2b08944 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -64,7 +64,7 @@ public class CandidateState implements EpochState { int electionTimeoutMs, LogContext logContext ) { - if (!voters.isVoter(ReplicaKey.of(localId, Optional.of(localDirectoryId)))) { + if (!voters.isVoter(ReplicaKey.of(localId, localDirectoryId))) { throw new IllegalArgumentException( String.format( "Local replica (%d, %s) must be in the set of voters %s", @@ -247,7 +247,7 @@ public class CandidateState implements EpochState { public ElectionState election() { return ElectionState.withVotedCandidate( epoch, - ReplicaKey.of(localId, Optional.of(localDirectoryId)), + ReplicaKey.of(localId, localDirectoryId), voteStates.keySet() ); } @@ -257,6 +257,11 @@ public class CandidateState implements EpochState { return epoch; } + @Override + public Endpoints leaderEndpoints() { + return Endpoints.empty(); + } + @Override public Optional highWatermark() { return highWatermark; diff --git a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java index 367f104cbfa..320ae47dbe9 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.raft; -import org.apache.kafka.common.Uuid; import org.apache.kafka.raft.generated.QuorumStateData; import org.apache.kafka.raft.internals.ReplicaKey; @@ -33,7 +32,6 @@ import java.util.stream.Collectors; public final class ElectionState { private static final int UNKNOWN_LEADER_ID = -1; private static final int NOT_VOTED = -1; - private static final Uuid NO_VOTED_DIRECTORY_ID = Uuid.ZERO_UUID; private final int epoch; private final OptionalInt leaderId; @@ -136,7 +134,9 @@ public final class ElectionState { .collect(Collectors.toList()); data.setCurrentVoters(dataVoters); } else if (version == 1) { - data.setVotedDirectoryId(votedKey.flatMap(ReplicaKey::directoryId).orElse(NO_VOTED_DIRECTORY_ID)); + data.setVotedDirectoryId( + votedKey.flatMap(ReplicaKey::directoryId).orElse(ReplicaKey.NO_DIRECTORY_ID) + ); } else { throw new IllegalStateException( String.format( @@ -199,13 +199,9 @@ public final class ElectionState { } public static ElectionState fromQuorumStateData(QuorumStateData data) { - Optional votedDirectoryId = data.votedDirectoryId().equals(NO_VOTED_DIRECTORY_ID) ? - Optional.empty() : - Optional.of(data.votedDirectoryId()); - Optional votedKey = data.votedId() == NOT_VOTED ? Optional.empty() : - Optional.of(ReplicaKey.of(data.votedId(), votedDirectoryId)); + Optional.of(ReplicaKey.of(data.votedId(), data.votedDirectoryId())); return new ElectionState( data.leaderEpoch(), diff --git a/raft/src/main/java/org/apache/kafka/raft/Endpoints.java b/raft/src/main/java/org/apache/kafka/raft/Endpoints.java new file mode 100644 index 00000000000..f3b0714215e --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/Endpoints.java @@ -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 endpoints; + + private Endpoints(Map endpoints) { + this.endpoints = endpoints; + } + + public Optional address(ListenerName listener) { + return Optional.ofNullable(endpoints.get(listener)); + } + + public Iterator 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 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 endpoints) { + return new Endpoints(endpoints); + } + + public static Endpoints fromVotersRecordEndpoints(Collection endpoints) { + Map 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 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 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 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 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 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 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 listeners = new HashMap<>(endpoints.size()); + for (FetchSnapshotResponseData.NodeEndpoint endpoint : endpoints) { + listeners.put( + listenerName, + InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port()) + ); + } + + return new Endpoints(listeners); + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index fc11a743b47..64642becce7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochState.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochState.java @@ -49,6 +49,13 @@ public interface EpochState extends Closeable { */ int epoch(); + /** + * Returns the known endpoints for the leader. + * + * If the leader is not known then {@code Endpoints.empty()} is returned. + */ + Endpoints leaderEndpoints(); + /** * User-friendly description of the state */ diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index bfbd257ece2..61004fe34f0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Node; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -32,7 +33,8 @@ import java.util.Set; public class FollowerState implements EpochState { private final int fetchTimeoutMs; private final int epoch; - private final Node leader; + private final int leaderId; + private final Endpoints endpoints; private final Set voters; // Used for tracking the expiration of both the Fetch and FetchSnapshot requests private final Timer fetchTimer; @@ -47,7 +49,8 @@ public class FollowerState implements EpochState { public FollowerState( Time time, int epoch, - Node leader, + int leaderId, + Endpoints endpoints, Set voters, Optional highWatermark, int fetchTimeoutMs, @@ -55,7 +58,8 @@ public class FollowerState implements EpochState { ) { this.fetchTimeoutMs = fetchTimeoutMs; this.epoch = epoch; - this.leader = leader; + this.leaderId = leaderId; + this.endpoints = endpoints; this.voters = voters; this.fetchTimer = time.timer(fetchTimeoutMs); this.highWatermark = highWatermark; @@ -64,7 +68,7 @@ public class FollowerState implements EpochState { @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, leader.id(), voters); + return ElectionState.withElectedLeader(epoch, leaderId, voters); } @Override @@ -72,6 +76,11 @@ public class FollowerState implements EpochState { return epoch; } + @Override + public Endpoints leaderEndpoints() { + return endpoints; + } + @Override public String name() { return "Follower"; @@ -82,8 +91,28 @@ public class FollowerState implements EpochState { return fetchTimer.remainingMs(); } - public Node leader() { - return leader; + public int leaderId() { + return leaderId; + } + + public Node leaderNode(ListenerName listener) { + /* KAFKA-16529 is going to change this so that the leader is not required to be in the set + * of voters. In other words, don't throw an IllegalStateException if the leader is not in + * the set of voters. + */ + return endpoints + .address(listener) + .map(address -> new Node(leaderId, address.getHostString(), address.getPort())) + .orElseThrow(() -> + new IllegalArgumentException( + String.format( + "Unknown endpoint for leader %d and listener %s, known endpoints are %s", + leaderId, + listener, + endpoints + ) + ) + ); } public boolean hasFetchTimeoutExpired(long currentTimeMs) { @@ -158,7 +187,7 @@ public class FollowerState implements EpochState { log.debug( "Rejecting vote request from candidate ({}) since we already have a leader {} in epoch {}", candidateKey, - leader, + leaderId, epoch ); return false; @@ -167,11 +196,12 @@ public class FollowerState implements EpochState { @Override public String toString() { return String.format( - "FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%s voters=%s, highWatermark=%s, " + + "FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%d, endpoints=%s, voters=%s, highWatermark=%s, " + "fetchingSnapshot=%s)", fetchTimeoutMs, epoch, - leader, + leaderId, + endpoints, voters, highWatermark, fetchingSnapshot diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index c2894def7f5..2e47eebadca 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -46,17 +46,14 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.record.UnalignedRecords; import org.apache.kafka.common.requests.BeginQuorumEpochRequest; -import org.apache.kafka.common.requests.BeginQuorumEpochResponse; import org.apache.kafka.common.requests.DescribeQuorumRequest; import org.apache.kafka.common.requests.DescribeQuorumResponse; import org.apache.kafka.common.requests.EndQuorumEpochRequest; -import org.apache.kafka.common.requests.EndQuorumEpochResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchSnapshotRequest; import org.apache.kafka.common.requests.FetchSnapshotResponse; import org.apache.kafka.common.requests.VoteRequest; -import org.apache.kafka.common.requests.VoteResponse; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -164,6 +161,7 @@ public final class KafkaRaftClient implements RaftClient { private final Time time; private final int fetchMaxWaitMs; private final String clusterId; + private final Endpoints localListeners; private final NetworkChannel channel; private final ReplicatedLog log; private final Random random; @@ -214,6 +212,7 @@ public final class KafkaRaftClient implements RaftClient { LogContext logContext, String clusterId, Collection bootstrapServers, + Endpoints localListeners, QuorumConfig quorumConfig ) { this( @@ -229,6 +228,7 @@ public final class KafkaRaftClient implements RaftClient { MAX_FETCH_WAIT_MS, clusterId, bootstrapServers, + localListeners, logContext, new Random(), quorumConfig @@ -248,6 +248,7 @@ public final class KafkaRaftClient implements RaftClient { int fetchMaxWaitMs, String clusterId, Collection bootstrapServers, + Endpoints localListeners, LogContext logContext, Random random, QuorumConfig quorumConfig @@ -264,6 +265,7 @@ public final class KafkaRaftClient implements RaftClient { this.appendPurgatory = new ThresholdPurgatory<>(expirationService); this.time = time; this.clusterId = clusterId; + this.localListeners = localListeners; this.fetchMaxWaitMs = fetchMaxWaitMs; this.logger = logContext.logger(KafkaRaftClient.class); this.random = random; @@ -315,7 +317,7 @@ public final class KafkaRaftClient implements RaftClient { ) { final LogOffsetMetadata endOffsetMetadata = log.endOffset(); - if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet().voters())) { + if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet())) { onUpdateLeaderHighWatermark(state, currentTimeMs); } @@ -452,9 +454,9 @@ public final class KafkaRaftClient implements RaftClient { quorum = new QuorumState( nodeId, nodeDirectoryId, - channel.listenerName(), partitionState::lastVoterSet, partitionState::lastKraftVersion, + localListeners, quorumConfig.electionTimeoutMs(), quorumConfig.fetchTimeoutMs(), quorumStateStore, @@ -613,22 +615,32 @@ public final class KafkaRaftClient implements RaftClient { private void transitionToFollower( int epoch, - Node leader, + int leaderId, + Endpoints endpoints, long currentTimeMs ) { - quorum.transitionToFollower(epoch, leader); + quorum.transitionToFollower(epoch, leaderId, endpoints); maybeFireLeaderChange(); onBecomeFollower(currentTimeMs); } - private VoteResponseData buildVoteResponse(Errors partitionLevelError, boolean voteGranted) { - return VoteResponse.singletonResponse( + private VoteResponseData buildVoteResponse( + ListenerName listenerName, + short apiVersion, + Errors partitionLevelError, + boolean voteGranted + ) { + return RaftUtil.singletonVoteResponse( + listenerName, + apiVersion, Errors.NONE, log.topicPartition(), partitionLevelError, quorum.epoch(), quorum.leaderIdOrSentinel(), - voteGranted); + voteGranted, + quorum.leaderEndpoints() + ); } /** @@ -638,8 +650,6 @@ public final class KafkaRaftClient implements RaftClient { * but different from this node * - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down * - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch - * - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g. - * if this node or the sender is not one of the current known voters) * - {@link Errors#INVALID_REQUEST} if the last epoch or offset are invalid */ private VoteResponseData handleVoteRequest( @@ -665,20 +675,52 @@ public final class KafkaRaftClient implements RaftClient { int lastEpoch = partitionRequest.lastOffsetEpoch(); long lastEpochEndOffset = partitionRequest.lastOffset(); if (lastEpochEndOffset < 0 || lastEpoch < 0 || lastEpoch >= candidateEpoch) { - return buildVoteResponse(Errors.INVALID_REQUEST, false); + return buildVoteResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.INVALID_REQUEST, + false + ); } Optional errorOpt = validateVoterOnlyRequest(candidateId, candidateEpoch); if (errorOpt.isPresent()) { - return buildVoteResponse(errorOpt.get(), false); + return buildVoteResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + errorOpt.get(), + false + ); } if (candidateEpoch > quorum.epoch()) { transitionToUnattached(candidateEpoch); } + // Check that the request was intended for this replica + Optional voterKey = RaftUtil.voteRequestVoterKey(request, partitionRequest); + if (!isValidVoterKey(voterKey)) { + logger.info( + "Candidate sent a voter key ({}) in the VOTE request that doesn't match the " + + "local key ({}, {}); rejecting the vote", + voterKey, + nodeId, + nodeDirectoryId + ); + // The request is not intended to this replica since the replica keys don't match + return buildVoteResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.INVALID_VOTER_KEY, + false + ); + } + OffsetAndEpoch lastEpochEndOffsetAndEpoch = new OffsetAndEpoch(lastEpochEndOffset, lastEpoch); - ReplicaKey candidateKey = ReplicaKey.of(candidateId, Optional.empty()); + ReplicaKey candidateKey = ReplicaKey.of( + candidateId, + partitionRequest.candidateDirectoryId() + ); boolean voteGranted = quorum.canGrantVote( candidateKey, lastEpochEndOffsetAndEpoch.compareTo(endOffset()) >= 0 @@ -689,7 +731,12 @@ public final class KafkaRaftClient implements RaftClient { } logger.info("Vote request {} with epoch {} is {}", request, candidateEpoch, voteGranted ? "granted" : "rejected"); - return buildVoteResponse(Errors.NONE, voteGranted); + return buildVoteResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.NONE, + voteGranted + ); } private boolean handleVoteResponse( @@ -716,6 +763,7 @@ public final class KafkaRaftClient implements RaftClient { Optional handled = maybeHandleCommonResponse( error, responseLeaderId, responseEpoch, currentTimeMs); + // KAFKA-16529 will need to handle the INVALID_VOTER_KEY error when handling the response if (handled.isPresent()) { return handled.get(); } else if (error == Errors.NONE) { @@ -763,22 +811,31 @@ public final class KafkaRaftClient implements RaftClient { } private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { - if (positionInSuccessors <= 0 || positionInSuccessors >= totalNumSuccessors) { - throw new IllegalArgumentException("Position " + positionInSuccessors + " should be larger than zero" + - " and smaller than total number of successors " + totalNumSuccessors); + if (positionInSuccessors == 0) { + return 0; + } else if (positionInSuccessors < 0 || positionInSuccessors >= totalNumSuccessors) { + return quorumConfig.electionBackoffMaxMs(); } int retryBackOffBaseMs = quorumConfig.electionBackoffMaxMs() >> (totalNumSuccessors - 1); return Math.min(quorumConfig.electionBackoffMaxMs(), retryBackOffBaseMs << (positionInSuccessors - 1)); } - private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse(Errors partitionLevelError) { - return BeginQuorumEpochResponse.singletonResponse( + private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse( + ListenerName listenerName, + short apiVersion, + Errors partitionLevelError + ) { + return RaftUtil.singletonBeginQuorumEpochResponse( + listenerName, + apiVersion, Errors.NONE, log.topicPartition(), partitionLevelError, quorum.epoch(), - quorum.leaderIdOrSentinel()); + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints() + ); } /** @@ -787,8 +844,6 @@ public final class KafkaRaftClient implements RaftClient { * - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request * but different from this node * - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down - * - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g. - * if this node or the sender is not one of the current known voters) * - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch */ private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest( @@ -814,15 +869,51 @@ public final class KafkaRaftClient implements RaftClient { Optional errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch); if (errorOpt.isPresent()) { - return buildBeginQuorumEpochResponse(errorOpt.get()); + return buildBeginQuorumEpochResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + errorOpt.get() + ); + } + + // Compute the leader's endpoint from the request or the voter set + final Endpoints leaderEndpoints; + if (request.leaderEndpoints().isEmpty()) { + leaderEndpoints = partitionState.lastVoterSet().listeners(requestLeaderId); + } else { + leaderEndpoints = Endpoints.fromBeginQuorumEpochRequest(request.leaderEndpoints()); } maybeTransition( - partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()), + OptionalInt.of(requestLeaderId), requestEpoch, + leaderEndpoints, currentTimeMs ); - return buildBeginQuorumEpochResponse(Errors.NONE); + + // Check that the request was intended for this replica + Optional voterKey = RaftUtil.beginQuorumEpochRequestVoterKey(request, partitionRequest); + if (!isValidVoterKey(voterKey)) { + logger.info( + "Leader sent a voter key ({}) in the BEGIN_QUORUM_EPOCH request that doesn't " + + "match the local key ({}, {}); returning INVALID_VOTER_KEY", + voterKey, + nodeId, + nodeDirectoryId + ); + // The request is not intended to this replica since the replica keys don't match + return buildBeginQuorumEpochResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.INVALID_VOTER_KEY + ); + } + + return buildBeginQuorumEpochResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.NONE + ); } private boolean handleBeginQuorumEpochResponse( @@ -849,6 +940,7 @@ public final class KafkaRaftClient implements RaftClient { Optional handled = maybeHandleCommonResponse( partitionError, responseLeaderId, responseEpoch, currentTimeMs); + // KAFKA-16529 will need to handle the INVALID_VOTER_KEY error when handling the response if (handled.isPresent()) { return handled.get(); } else if (partitionError == Errors.NONE) { @@ -865,13 +957,21 @@ public final class KafkaRaftClient implements RaftClient { } } - private EndQuorumEpochResponseData buildEndQuorumEpochResponse(Errors partitionLevelError) { - return EndQuorumEpochResponse.singletonResponse( + private EndQuorumEpochResponseData buildEndQuorumEpochResponse( + ListenerName listenerName, + short apiVersion, + Errors partitionLevelError + ) { + return RaftUtil.singletonEndQuorumEpochResponse( + listenerName, + apiVersion, Errors.NONE, log.topicPartition(), partitionLevelError, quorum.epoch(), - quorum.leaderIdOrSentinel()); + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints() + ); } /** @@ -880,8 +980,6 @@ public final class KafkaRaftClient implements RaftClient { * - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request * but different from this node * - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down - * - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g. - * if this node or the sender is not one of the current known voters) * - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch */ private EndQuorumEpochResponseData handleEndQuorumEpochRequest( @@ -907,39 +1005,72 @@ public final class KafkaRaftClient implements RaftClient { Optional errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch); if (errorOpt.isPresent()) { - return buildEndQuorumEpochResponse(errorOpt.get()); + return buildEndQuorumEpochResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + errorOpt.get() + ); } + + final Endpoints leaderEndpoints; + if (request.leaderEndpoints().isEmpty()) { + leaderEndpoints = partitionState.lastVoterSet().listeners(requestLeaderId); + } else { + leaderEndpoints = Endpoints.fromEndQuorumEpochRequest(request.leaderEndpoints()); + } + maybeTransition( - partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()), + OptionalInt.of(requestLeaderId), requestEpoch, + leaderEndpoints, currentTimeMs ); if (quorum.isFollower()) { FollowerState state = quorum.followerStateOrThrow(); - if (state.leader().id() == requestLeaderId) { - List preferredSuccessors = partitionRequest.preferredSuccessors(); - long electionBackoffMs = endEpochElectionBackoff(preferredSuccessors); - logger.debug("Overriding follower fetch timeout to {} after receiving " + - "EndQuorumEpoch request from leader {} in epoch {}", electionBackoffMs, - requestLeaderId, requestEpoch); + if (state.leaderId() == requestLeaderId) { + List preferredCandidates = EndQuorumEpochRequest + .preferredCandidates(partitionRequest) + .stream() + .map(replica -> ReplicaKey.of(replica.candidateId(), replica.candidateDirectoryId())) + .collect(Collectors.toList()); + long electionBackoffMs = endEpochElectionBackoff(preferredCandidates); + logger.debug( + "Overriding follower fetch timeout to {} after receiving EndQuorumEpoch " + + "request from leader {} in epoch {}", + electionBackoffMs, + requestLeaderId, + requestEpoch + ); state.overrideFetchTimeout(currentTimeMs, electionBackoffMs); } } - return buildEndQuorumEpochResponse(Errors.NONE); + return buildEndQuorumEpochResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.NONE + ); } - private long endEpochElectionBackoff(List preferredSuccessors) { + private long endEpochElectionBackoff(Collection preferredCandidates) { // Based on the priority inside the preferred successors, choose the corresponding delayed // election backoff time based on strict exponential mechanism so that the most up-to-date // voter has a higher chance to be elected. If the node's priority is highest, become // candidate immediately instead of waiting for next poll. - int position = preferredSuccessors.indexOf(quorum.localIdOrThrow()); - if (position <= 0) { - return 0; - } else { - return strictExponentialElectionBackoffMs(position, preferredSuccessors.size()); + int position = 0; + for (ReplicaKey candidate : preferredCandidates) { + if (candidate.id() == quorum.localIdOrThrow()) { + if (!candidate.directoryId().isPresent() || + candidate.directoryId().get().equals(quorum.localDirectoryId()) + ) { + // Found ourselves in the preferred candidate list + break; + } + } + position++; } + + return strictExponentialElectionBackoffMs(position, preferredCandidates.size()); } private boolean handleEndQuorumEpochResponse( @@ -977,45 +1108,60 @@ public final class KafkaRaftClient implements RaftClient { } private FetchResponseData buildFetchResponse( + ListenerName listenerName, + short apiVersion, Errors error, Records records, ValidOffsetAndEpoch validOffsetAndEpoch, Optional highWatermark ) { - return RaftUtil.singletonFetchResponse(log.topicPartition(), log.topicId(), Errors.NONE, partitionData -> { - partitionData - .setRecords(records) - .setErrorCode(error.code()) - .setLogStartOffset(log.startOffset()) - .setHighWatermark( - highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L) - ); + return RaftUtil.singletonFetchResponse( + listenerName, + apiVersion, + log.topicPartition(), + log.topicId(), + Errors.NONE, + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints(), + partitionData -> { + partitionData + .setRecords(records) + .setErrorCode(error.code()) + .setLogStartOffset(log.startOffset()) + .setHighWatermark( + highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L) + ); - partitionData.currentLeader() - .setLeaderEpoch(quorum.epoch()) - .setLeaderId(quorum.leaderIdOrSentinel()); + partitionData.currentLeader() + .setLeaderEpoch(quorum.epoch()) + .setLeaderId(quorum.leaderIdOrSentinel()); - switch (validOffsetAndEpoch.kind()) { - case DIVERGING: - partitionData.divergingEpoch() - .setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch()) - .setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset()); - break; - case SNAPSHOT: - partitionData.snapshotId() - .setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch()) - .setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset()); - break; - default: + switch (validOffsetAndEpoch.kind()) { + case DIVERGING: + partitionData.divergingEpoch() + .setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch()) + .setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset()); + break; + case SNAPSHOT: + partitionData.snapshotId() + .setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch()) + .setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset()); + break; + default: + } } - }); + ); } private FetchResponseData buildEmptyFetchResponse( + ListenerName listenerName, + short apiVersion, Errors error, Optional highWatermark ) { return buildFetchResponse( + listenerName, + apiVersion, error, MemoryRecords.EMPTY, ValidOffsetAndEpoch.valid(), @@ -1069,12 +1215,26 @@ public final class KafkaRaftClient implements RaftClient { || fetchPartition.lastFetchedEpoch() < 0 || fetchPartition.lastFetchedEpoch() > fetchPartition.currentLeaderEpoch()) { return completedFuture( - buildEmptyFetchResponse(Errors.INVALID_REQUEST, Optional.empty()) + buildEmptyFetchResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + Errors.INVALID_REQUEST, + Optional.empty() + ) ); } - int replicaId = FetchRequest.replicaId(request); - FetchResponseData response = tryCompleteFetchRequest(replicaId, fetchPartition, currentTimeMs); + ReplicaKey replicaKey = ReplicaKey.of( + FetchRequest.replicaId(request), + fetchPartition.replicaDirectoryId() + ); + FetchResponseData response = tryCompleteFetchRequest( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + replicaKey, + fetchPartition, + currentTimeMs + ); FetchResponseData.PartitionData partitionResponse = response.responses().get(0).partitions().get(0); @@ -1111,32 +1271,53 @@ public final class KafkaRaftClient implements RaftClient { return response; } else { // If there was any error other than REQUEST_TIMED_OUT, return it. - logger.info("Failed to handle fetch from {} at {} due to {}", - replicaId, fetchPartition.fetchOffset(), error); - return buildEmptyFetchResponse(error, Optional.empty()); + logger.info( + "Failed to handle fetch from {} at {} due to {}", + replicaKey, + fetchPartition.fetchOffset(), + error + ); + return buildEmptyFetchResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + error, + Optional.empty() + ); } } // FIXME: `completionTimeMs`, which can be null - logger.trace("Completing delayed fetch from {} starting at offset {} at {}", - replicaId, fetchPartition.fetchOffset(), completionTimeMs); + logger.trace( + "Completing delayed fetch from {} starting at offset {} at {}", + replicaKey, + fetchPartition.fetchOffset(), + completionTimeMs + ); // It is safe to call tryCompleteFetchRequest because only the polling thread completes this // future successfully. This is true because only the polling thread appends record batches to // the log from maybeAppendBatches. - return tryCompleteFetchRequest(replicaId, fetchPartition, time.milliseconds()); + return tryCompleteFetchRequest( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), + replicaKey, + fetchPartition, + time.milliseconds() + ); }); } private FetchResponseData tryCompleteFetchRequest( - int replicaId, + ListenerName listenerName, + short apiVersion, + ReplicaKey replicaKey, FetchRequestData.FetchPartition request, long currentTimeMs ) { try { Optional errorOpt = validateLeaderOnlyRequest(request.currentLeaderEpoch()); if (errorOpt.isPresent()) { - return buildEmptyFetchResponse(errorOpt.get(), Optional.empty()); + return buildEmptyFetchResponse(listenerName, apiVersion, errorOpt.get(), Optional.empty()); } long fetchOffset = request.fetchOffset(); @@ -1157,7 +1338,7 @@ public final class KafkaRaftClient implements RaftClient { if (validOffsetAndEpoch.kind() == ValidOffsetAndEpoch.Kind.VALID) { LogFetchInfo info = log.read(fetchOffset, Isolation.UNCOMMITTED); - if (state.updateReplicaState(replicaId, Uuid.ZERO_UUID, currentTimeMs, info.startOffsetMetadata)) { + if (state.updateReplicaState(replicaKey, currentTimeMs, info.startOffsetMetadata)) { onUpdateLeaderHighWatermark(state, currentTimeMs); } @@ -1166,10 +1347,17 @@ public final class KafkaRaftClient implements RaftClient { records = MemoryRecords.EMPTY; } - return buildFetchResponse(Errors.NONE, records, validOffsetAndEpoch, state.highWatermark()); + return buildFetchResponse( + listenerName, + apiVersion, + Errors.NONE, + records, + validOffsetAndEpoch, + state.highWatermark() + ); } catch (Exception e) { logger.error("Caught unexpected error in fetch completion of request {}", request, e); - return buildEmptyFetchResponse(Errors.UNKNOWN_SERVER_ERROR, Optional.empty()); + return buildEmptyFetchResponse(listenerName, apiVersion, Errors.UNKNOWN_SERVER_ERROR, Optional.empty()); } } @@ -1352,12 +1540,11 @@ public final class KafkaRaftClient implements RaftClient { } LeaderState leaderState = quorum.leaderStateOrThrow(); - return DescribeQuorumResponse.singletonResponse( + return RaftUtil.singletonDescribeQuorumResponse( + requestMetadata.apiVersion(), log.topicPartition(), leaderState.describeQuorum(currentTimeMs), - requestMetadata.apiVersion() < DescribeQuorumResponseData.Node.LOWEST_SUPPORTED_VERSION - ? null - : leaderState.nodes(currentTimeMs) + leaderState.nodes(currentTimeMs) ); } @@ -1399,8 +1586,12 @@ public final class KafkaRaftClient implements RaftClient { data.topics().get(0).partitions().get(0).partition() ); - return FetchSnapshotResponse.singleton( + return RaftUtil.singletonFetchSnapshotResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), unknownTopicPartition, + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints(), responsePartitionSnapshot -> responsePartitionSnapshot .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) ); @@ -1411,8 +1602,12 @@ public final class KafkaRaftClient implements RaftClient { partitionSnapshot.currentLeaderEpoch() ); if (leaderValidation.isPresent()) { - return FetchSnapshotResponse.singleton( + return RaftUtil.singletonFetchSnapshotResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), log.topicPartition(), + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints(), responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) .setErrorCode(leaderValidation.get().code()) ); @@ -1424,8 +1619,12 @@ public final class KafkaRaftClient implements RaftClient { ); Optional snapshotOpt = log.readSnapshot(snapshotId); if (!snapshotOpt.isPresent()) { - return FetchSnapshotResponse.singleton( + return RaftUtil.singletonFetchSnapshotResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), log.topicPartition(), + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints(), responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) .setErrorCode(Errors.SNAPSHOT_NOT_FOUND.code()) ); @@ -1434,8 +1633,12 @@ public final class KafkaRaftClient implements RaftClient { RawSnapshotReader snapshot = snapshotOpt.get(); long snapshotSize = snapshot.sizeInBytes(); if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshotSize) { - return FetchSnapshotResponse.singleton( + return RaftUtil.singletonFetchSnapshotResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), log.topicPartition(), + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints(), responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) .setErrorCode(Errors.POSITION_OUT_OF_RANGE.code()) ); @@ -1462,10 +1665,20 @@ public final class KafkaRaftClient implements RaftClient { UnalignedRecords records = snapshot.slice(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize)); LeaderState state = quorum.leaderStateOrThrow(); - state.updateCheckQuorumForFollowingVoter(data.replicaId(), currentTimeMs); + state.updateCheckQuorumForFollowingVoter( + ReplicaKey.of( + data.replicaId(), + partitionSnapshot.replicaDirectoryId() + ), + currentTimeMs + ); - return FetchSnapshotResponse.singleton( + return RaftUtil.singletonFetchSnapshotResponse( + requestMetadata.listenerName(), + requestMetadata.apiVersion(), log.topicPartition(), + quorum.leaderIdOrSentinel(), + quorum.leaderEndpoints(), responsePartitionSnapshot -> { addQuorumLeader(responsePartitionSnapshot) .snapshotId() @@ -1648,11 +1861,11 @@ public final class KafkaRaftClient implements RaftClient { int epoch, long currentTimeMs ) { - Optional leader = leaderId.isPresent() ? - partitionState + Endpoints leaderEndpoints = leaderId.isPresent() ? + leaderEndpoints = partitionState .lastVoterSet() - .voterNode(leaderId.getAsInt(), channel.listenerName()) : - Optional.empty(); + .listeners(leaderId.getAsInt()) : + Endpoints.empty(); if (epoch < quorum.epoch() || error == Errors.UNKNOWN_LEADER_EPOCH) { // We have a larger epoch, so the response is no longer relevant @@ -1663,10 +1876,10 @@ public final class KafkaRaftClient implements RaftClient { // The response indicates that the request had a stale epoch, but we need // to validate the epoch from the response against our current state. - maybeTransition(leader, epoch, currentTimeMs); + maybeTransition(leaderId, epoch, leaderEndpoints, currentTimeMs); return Optional.of(true); } else if (epoch == quorum.epoch() - && leader.isPresent() + && leaderId.isPresent() && !quorum.hasLeader()) { // Since we are transitioning to Follower, we will only forward the @@ -1674,7 +1887,7 @@ public final class KafkaRaftClient implements RaftClient { // the request be retried immediately (if needed) after the transition. // This handling allows an observer to discover the leader and append // to the log in the same Fetch request. - transitionToFollower(epoch, leader.get(), currentTimeMs); + transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs); if (error == Errors.NONE) { return Optional.empty(); } else { @@ -1696,28 +1909,28 @@ public final class KafkaRaftClient implements RaftClient { } private void maybeTransition( - Optional leader, + OptionalInt leaderId, int epoch, + Endpoints leaderEndpoints, long currentTimeMs ) { - OptionalInt leaderId = leader.isPresent() ? - OptionalInt.of(leader.get().id()) : - OptionalInt.empty(); - if (!hasConsistentLeader(epoch, leaderId)) { - throw new IllegalStateException("Received request or response with leader " + leader + + throw new IllegalStateException("Received request or response with leader " + leaderId + " and epoch " + epoch + " which is inconsistent with current leader " + quorum.leaderId() + " and epoch " + quorum.epoch()); } else if (epoch > quorum.epoch()) { - if (leader.isPresent()) { - transitionToFollower(epoch, leader.get(), currentTimeMs); + if (leaderId.isPresent()) { + transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs); } else { transitionToUnattached(epoch); } - } else if (leader.isPresent() && !quorum.hasLeader()) { - // The request or response indicates the leader of the current epoch, - // which is currently unknown - transitionToFollower(epoch, leader.get(), currentTimeMs); + } else if ( + leaderId.isPresent() && + (!quorum.hasLeader() || leaderEndpoints.size() > quorum.leaderEndpoints().size()) + ) { + // The request or response indicates the leader of the current epoch + // which are currently unknown or the replica has discovered more endpoints + transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs); } } @@ -1791,6 +2004,17 @@ public final class KafkaRaftClient implements RaftClient { } } + /** + * Return true if the voter key matches the local replica's key + */ + private boolean isValidVoterKey(Optional voterKey) { + return voterKey + .map(key -> + OptionalInt.of(key.id()).equals(nodeId) && + key.directoryId().equals(Optional.of(nodeDirectoryId)) + ) + .orElse(true); + } /** * Validate a request which is intended for the current quorum leader. * If an error is present in the returned value, it should be returned @@ -1847,10 +2071,8 @@ public final class KafkaRaftClient implements RaftClient { } responseFuture.whenComplete((response, exception) -> { - final ApiMessage message; - if (response != null) { - message = response; - } else { + ApiMessage message = response; + if (message == null) { message = RaftUtil.errorResponse(apiKey, Errors.forException(exception)); } @@ -1898,7 +2120,6 @@ public final class KafkaRaftClient implements RaftClient { RaftRequest.Outbound requestMessage = new RaftRequest.Outbound( correlationId, - request.highestSupportedVersion(), request, destination, currentTimeMs @@ -2231,11 +2452,12 @@ public final class KafkaRaftClient implements RaftClient { // If the current leader is backing off due to some failure or if the // request has timed out, then we attempt to send the Fetch to another // voter in order to discover if there has been a leader change. - if (requestManager.hasRequestTimedOut(state.leader(), currentTimeMs)) { + Node leaderNode = state.leaderNode(channel.listenerName()); + if (requestManager.hasRequestTimedOut(leaderNode, currentTimeMs)) { // Once the request has timed out backoff the connection - requestManager.reset(state.leader()); + requestManager.reset(leaderNode); backoffMs = maybeSendAnyVoterFetch(currentTimeMs); - } else if (requestManager.isBackingOff(state.leader(), currentTimeMs)) { + } else if (requestManager.isBackingOff(leaderNode, currentTimeMs)) { backoffMs = maybeSendAnyVoterFetch(currentTimeMs); } else if (!requestManager.hasAnyInflightRequest(currentTimeMs)) { backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs); @@ -2259,7 +2481,11 @@ public final class KafkaRaftClient implements RaftClient { requestSupplier = this::buildFetchRequest; } - return maybeSendRequest(currentTimeMs, state.leader(), requestSupplier); + return maybeSendRequest( + currentTimeMs, + state.leaderNode(channel.listenerName()), + requestSupplier + ); } private long pollVoted(long currentTimeMs) { diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java index 710c8b102f2..2af7d7d207f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.server.fault.FaultHandler; @@ -101,11 +102,13 @@ public class KafkaRaftClientDriver extends ShutdownableThread { } public CompletableFuture handleRequest( + RequestContext context, RequestHeader header, ApiMessage request, long createdTimeMs ) { RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound( + context.listenerName, header.correlationId(), header.apiVersion(), request, diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 7f0f6777453..031e3c8e14e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.raft; -import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; @@ -36,13 +35,13 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * In the context of LeaderState, an acknowledged voter means one who has acknowledged the current leader by either @@ -54,15 +53,15 @@ public class LeaderState implements EpochState { static final long OBSERVER_SESSION_TIMEOUT_MS = 300_000L; static final double CHECK_QUORUM_TIMEOUT_FACTOR = 1.5; - private final int localId; - private final Uuid localDirectoryId; + private final ReplicaKey localReplicaKey; private final int epoch; private final long epochStartOffset; private final Set grantingVoters; + private final Endpoints endpoints; private Optional highWatermark = Optional.empty(); - private final Map voterStates = new HashMap<>(); - private final Map observerStates = new HashMap<>(); + private Map voterStates = new HashMap<>(); + private final Map observerStates = new HashMap<>(); private final Logger log; private final BatchAccumulator accumulator; // The set includes all of the followers voters that FETCH or FETCH_SNAPSHOT during the current checkQuorumTimer interval. @@ -75,24 +74,27 @@ public class LeaderState implements EpochState { protected LeaderState( Time time, - int localId, - Uuid localDirectoryId, + ReplicaKey localReplicaKey, int epoch, long epochStartOffset, - Map voters, + VoterSet voters, Set grantingVoters, BatchAccumulator accumulator, + Endpoints endpoints, int fetchTimeoutMs, LogContext logContext ) { - this.localId = localId; - this.localDirectoryId = localDirectoryId; + this.localReplicaKey = localReplicaKey; this.epoch = epoch; this.epochStartOffset = epochStartOffset; + this.endpoints = endpoints; - for (Map.Entry voter : voters.entrySet()) { - boolean hasAcknowledgedLeader = voter.getKey() == localId; - this.voterStates.put(voter.getKey(), new ReplicaState(voter.getKey(), voter.getValue().voterKey().directoryId(), hasAcknowledgedLeader)); + for (VoterSet.VoterNode voterNode: voters.voterNodes()) { + boolean hasAcknowledgedLeader = voterNode.isVoter(localReplicaKey); + this.voterStates.put( + voterNode.voterKey().id(), + new ReplicaState(voterNode.voterKey(), hasAcknowledgedLeader) + ); } this.grantingVoters = Collections.unmodifiableSet(new HashSet<>(grantingVoters)); this.log = logContext.logger(LeaderState.class); @@ -104,7 +106,9 @@ public class LeaderState implements EpochState { /** * Get the remaining time in milliseconds until the checkQuorumTimer expires. - * This will happen if we didn't receive a valid fetch/fetchSnapshot request from the majority of the voters within checkQuorumTimeoutMs. + * + * This will happen if we didn't receive a valid fetch/fetchSnapshot request from the majority + * of the voters within checkQuorumTimeoutMs. * * @param currentTimeMs the current timestamp in millisecond * @return the remainingMs before the checkQuorumTimer expired @@ -118,10 +122,12 @@ public class LeaderState implements EpochState { long remainingMs = checkQuorumTimer.remainingMs(); if (remainingMs == 0) { log.info( - "Did not receive fetch request from the majority of the voters within {}ms. Current fetched voters are {}, and voters are {}", + "Did not receive fetch request from the majority of the voters within {}ms. " + + "Current fetched voters are {}, and voters are {}", checkQuorumTimeoutMs, fetchedVoters, - voterStates.keySet()); + voterStates.values().stream().map(voter -> voter.replicaKey) + ); } return remainingMs; } @@ -132,15 +138,15 @@ public class LeaderState implements EpochState { * @param id the node id * @param currentTimeMs the current timestamp in millisecond */ - public void updateCheckQuorumForFollowingVoter(int id, long currentTimeMs) { - updateFetchedVoters(id); + public void updateCheckQuorumForFollowingVoter(ReplicaKey replicaKey, long currentTimeMs) { + updateFetchedVoters(replicaKey); // The majority number of the voters. Ex: 2 for 3 voters, 3 for 4 voters... etc. int majority = (voterStates.size() / 2) + 1; // If the leader is in the voter set, it should be implicitly counted as part of the // majority, but the leader will never be a member of the fetchedVoters. // If the leader is not in the voter set, it is not in the majority. Then, the // majority can only be composed of fetched voters. - if (voterStates.containsKey(localId)) { + if (voterStates.containsKey(localReplicaKey.id())) { majority = majority - 1; } @@ -151,13 +157,14 @@ public class LeaderState implements EpochState { } } - private void updateFetchedVoters(int id) { - if (id == localId) { + private void updateFetchedVoters(ReplicaKey replicaKey) { + if (replicaKey.id() == localReplicaKey.id()) { throw new IllegalArgumentException("Received a FETCH/FETCH_SNAPSHOT request from the leader itself."); } - if (isVoter(id)) { - fetchedVoters.add(id); + ReplicaState state = voterStates.get(replicaKey.id()); + if (state != null && state.matchesKey(replicaKey)) { + fetchedVoters.add(replicaKey.id()); } } @@ -200,7 +207,7 @@ public class LeaderState implements EpochState { @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, localId, voterStates.keySet()); + return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), voterStates.keySet()); } @Override @@ -208,30 +215,28 @@ public class LeaderState implements EpochState { return epoch; } + @Override + public Endpoints leaderEndpoints() { + return endpoints; + } + public Set grantingVoters() { return this.grantingVoters; } - public int localId() { - return localId; - } - - public Uuid localDirectoryId() { - return localDirectoryId; - } - public Set nonAcknowledgingVoters() { Set nonAcknowledging = new HashSet<>(); for (ReplicaState state : voterStates.values()) { if (!state.hasAcknowledgedLeader) - nonAcknowledging.add(state.nodeId); + nonAcknowledging.add(state.replicaKey.id()); } return nonAcknowledging; } private boolean maybeUpdateHighWatermark() { // Find the largest offset which is replicated to a majority of replicas (the leader counts) - List followersByDescendingFetchOffset = followersByDescendingFetchOffset(); + ArrayList followersByDescendingFetchOffset = followersByDescendingFetchOffset() + .collect(Collectors.toCollection(ArrayList::new)); int indexOfHw = voterStates.size() / 2; Optional highWatermarkUpdateOpt = followersByDescendingFetchOffset.get(indexOfHw).endOffset; @@ -323,17 +328,19 @@ public class LeaderState implements EpochState { */ public boolean updateLocalState( LogOffsetMetadata endOffsetMetadata, - Map lastVoters + VoterSet lastVoterSet ) { - ReplicaState state = getOrCreateReplicaState(localId, localDirectoryId); + ReplicaState state = getOrCreateReplicaState(localReplicaKey); state.endOffset.ifPresent(currentEndOffset -> { if (currentEndOffset.offset > endOffsetMetadata.offset) { throw new IllegalStateException("Detected non-monotonic update of local " + "end offset: " + currentEndOffset.offset + " -> " + endOffsetMetadata.offset); } }); + state.updateLeaderEndOffset(endOffsetMetadata); - updateVoterAndObserverStates(lastVoters); + updateVoterAndObserverStates(lastVoterSet); + return maybeUpdateHighWatermark(); } @@ -347,50 +354,53 @@ public class LeaderState implements EpochState { * @return true if the high watermark is updated as a result of this call */ public boolean updateReplicaState( - int replicaId, - Uuid replicaDirectoryId, + ReplicaKey replicaKey, long currentTimeMs, LogOffsetMetadata fetchOffsetMetadata ) { // Ignore fetches from negative replica id, as it indicates // the fetch is from non-replica. For example, a consumer. - if (replicaId < 0) { + if (replicaKey.id() < 0) { return false; - } else if (replicaId == localId) { - throw new IllegalStateException("Remote replica ID " + replicaId + " matches the local leader ID"); + } else if (replicaKey.id() == localReplicaKey.id()) { + throw new IllegalStateException( + String.format("Remote replica ID %s matches the local leader ID", replicaKey) + ); } - ReplicaState state = getOrCreateReplicaState(replicaId, replicaDirectoryId); + ReplicaState state = getOrCreateReplicaState(replicaKey); state.endOffset.ifPresent(currentEndOffset -> { if (currentEndOffset.offset > fetchOffsetMetadata.offset) { log.warn("Detected non-monotonic update of fetch offset from nodeId {}: {} -> {}", - state.nodeId, currentEndOffset.offset, fetchOffsetMetadata.offset); + state.replicaKey, currentEndOffset.offset, fetchOffsetMetadata.offset); } }); - Optional leaderEndOffsetOpt = getOrCreateReplicaState(localId, localDirectoryId).endOffset; + + Optional leaderEndOffsetOpt = getOrCreateReplicaState(localReplicaKey).endOffset; state.updateFollowerState( currentTimeMs, fetchOffsetMetadata, leaderEndOffsetOpt ); - updateCheckQuorumForFollowingVoter(replicaId, currentTimeMs); + updateCheckQuorumForFollowingVoter(replicaKey, currentTimeMs); - return isVoter(state.nodeId) && maybeUpdateHighWatermark(); + return isVoter(state.replicaKey) && maybeUpdateHighWatermark(); } public List nonLeaderVotersByDescendingFetchOffset() { - return followersByDescendingFetchOffset().stream() - .filter(state -> state.nodeId != localId) - .map(state -> state.nodeId) + return followersByDescendingFetchOffset() + .filter(state -> !state.matchesKey(localReplicaKey)) + .map(state -> state.replicaKey.id()) .collect(Collectors.toList()); } - private List followersByDescendingFetchOffset() { - return new ArrayList<>(this.voterStates.values()).stream() - .sorted() - .collect(Collectors.toList()); + private Stream followersByDescendingFetchOffset() { + return voterStates + .values() + .stream() + .sorted(); } public void addAcknowledgementFrom(int remoteNodeId) { @@ -400,8 +410,9 @@ public class LeaderState implements EpochState { private ReplicaState ensureValidVoter(int remoteNodeId) { ReplicaState state = voterStates.get(remoteNodeId); - if (state == null) + if (state == null) { throw new IllegalArgumentException("Unexpected acknowledgement from non-voter " + remoteNodeId); + } return state; } @@ -409,55 +420,58 @@ public class LeaderState implements EpochState { return epochStartOffset; } - private ReplicaState getOrCreateReplicaState(int remoteNodeId, Uuid remoteNodeDirectory) { - ReplicaState state = voterStates.get(remoteNodeId); - if (state == null) { - observerStates.putIfAbsent(remoteNodeId, new ReplicaState(remoteNodeId, Optional.of(remoteNodeDirectory), false)); - return observerStates.get(remoteNodeId); + private ReplicaState getOrCreateReplicaState(ReplicaKey replicaKey) { + ReplicaState state = voterStates.get(replicaKey.id()); + if (state == null || !state.matchesKey(replicaKey)) { + observerStates.putIfAbsent(replicaKey, new ReplicaState(replicaKey, false)); + return observerStates.get(replicaKey); } return state; } + private Optional getReplicaState(ReplicaKey replicaKey) { + ReplicaState state = voterStates.get(replicaKey.id()); + if (state == null || !state.matchesKey(replicaKey)) { + state = observerStates.get(replicaKey); + } + + return Optional.ofNullable(state); + } + public DescribeQuorumResponseData.PartitionData describeQuorum(long currentTimeMs) { clearInactiveObservers(currentTimeMs); return new DescribeQuorumResponseData.PartitionData() .setErrorCode(Errors.NONE.code()) - .setLeaderId(localId) + .setLeaderId(localReplicaKey.id()) .setLeaderEpoch(epoch) .setHighWatermark(highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L)) - .setCurrentVoters(describeReplicaStates(voterStates, currentTimeMs)) - .setObservers(describeReplicaStates(observerStates, currentTimeMs)); + .setCurrentVoters(describeReplicaStates(voterStates.values(), currentTimeMs)) + .setObservers(describeReplicaStates(observerStates.values(), currentTimeMs)); } public DescribeQuorumResponseData.NodeCollection nodes(long currentTimeMs) { clearInactiveObservers(currentTimeMs); - return nodes(voterStates.values(), observerStates.values()); - } + DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection(); - private static DescribeQuorumResponseData.NodeCollection nodes(Collection voters, Collection observers) { - DescribeQuorumResponseData.NodeCollection res = new DescribeQuorumResponseData.NodeCollection(); + voterStates.values().forEach(replicaState -> { + if (nodes.find(replicaState.replicaKey.id()) == null) { + // KAFKA-16953 will add support for including the node listeners in the node + // collection + nodes.add(new DescribeQuorumResponseData.Node().setNodeId(replicaState.replicaKey.id())); + } + }); - voters.forEach(replicaState -> node(res, replicaState)); - observers.forEach(replicaState -> node(res, replicaState)); - - return res; - } - - private static void node(DescribeQuorumResponseData.NodeCollection res, ReplicaState replicaState) { - if (res.find(replicaState.nodeId) != null) { - return; - } - - res.add(new DescribeQuorumResponseData.Node().setNodeId(replicaState.nodeId)); + return nodes; } private List describeReplicaStates( - Map state, + Collection states, long currentTimeMs ) { - return state.values().stream() + return states + .stream() .map(replicaState -> describeReplicaState(replicaState, currentTimeMs)) .collect(Collectors.toList()); } @@ -468,7 +482,7 @@ public class LeaderState implements EpochState { ) { final long lastCaughtUpTimestamp; final long lastFetchTimestamp; - if (replicaState.nodeId == localId) { + if (replicaState.matchesKey(localReplicaKey)) { lastCaughtUpTimestamp = currentTimeMs; lastFetchTimestamp = currentTimeMs; } else { @@ -476,8 +490,8 @@ public class LeaderState implements EpochState { lastFetchTimestamp = replicaState.lastFetchTimestamp; } return new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(replicaState.nodeId) - .setReplicaDirectoryId(replicaState.nodeDirectory.orElse(Uuid.ZERO_UUID)) + .setReplicaId(replicaState.replicaKey.id()) + .setReplicaDirectoryId(replicaState.replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(replicaState.endOffset.map(md -> md.offset).orElse(-1L)) .setLastCaughtUpTimestamp(lastCaughtUpTimestamp) .setLastFetchTimestamp(lastFetchTimestamp); @@ -490,46 +504,50 @@ public class LeaderState implements EpochState { private void clearInactiveObservers(final long currentTimeMs) { observerStates.entrySet().removeIf(integerReplicaStateEntry -> currentTimeMs - integerReplicaStateEntry.getValue().lastFetchTimestamp >= OBSERVER_SESSION_TIMEOUT_MS && - integerReplicaStateEntry.getKey() != localId + !integerReplicaStateEntry.getKey().equals(localReplicaKey) ); } - private boolean isVoter(int remoteNodeId) { - return voterStates.containsKey(remoteNodeId); + private boolean isVoter(ReplicaKey remoteReplicaKey) { + ReplicaState state = voterStates.get(remoteReplicaKey.id()); + return state != null && state.matchesKey(remoteReplicaKey); } - private void updateVoterAndObserverStates(Map lastVoters) { - // Move any replica that is not in the last voter set from voterStates to observerStates - for (Iterator> iter = voterStates.entrySet().iterator(); iter.hasNext(); ) { - Map.Entry replica = iter.next(); - if (!lastVoters.containsKey(replica.getKey())) { - observerStates.put(replica.getKey(), replica.getValue()); - iter.remove(); - } - } + private void updateVoterAndObserverStates(VoterSet lastVoterSet) { + Map newVoterStates = new HashMap<>(); + Map oldVoterStates = new HashMap<>(voterStates); - // Add replicas that are in the last voter set and not in voterStates to voterStates (from observerStates - // if they exist) - for (Map.Entry voter : lastVoters.entrySet()) { - if (!voterStates.containsKey(voter.getKey())) { - Optional existingObserverState = Optional.ofNullable(observerStates.remove(voter.getKey())); - voterStates.put(voter.getKey(), existingObserverState.orElse(new ReplicaState(voter.getKey(), voter.getValue().voterKey().directoryId(), false))); - } + // Compute the new voter states map + for (VoterSet.VoterNode voterNode : lastVoterSet.voterNodes()) { + ReplicaState state = getReplicaState(voterNode.voterKey()) + .orElse(new ReplicaState(voterNode.voterKey(), false)); + + // Remove the voter from the previous data structures + oldVoterStates.remove(voterNode.voterKey().id()); + observerStates.remove(voterNode.voterKey()); + + // Make sure that the replica key in the replica state matches the voter's + state.setReplicaKey(voterNode.voterKey()); + newVoterStates.put(state.replicaKey.id(), state); + } + voterStates = newVoterStates; + + // Move any of the remaining old voters to observerStates + for (ReplicaState replicaStateEntry : oldVoterStates.values()) { + observerStates.putIfAbsent(replicaStateEntry.replicaKey, replicaStateEntry); } } private static class ReplicaState implements Comparable { - final int nodeId; - final Optional nodeDirectory; + ReplicaKey replicaKey; Optional endOffset; long lastFetchTimestamp; long lastFetchLeaderLogEndOffset; long lastCaughtUpTimestamp; boolean hasAcknowledgedLeader; - public ReplicaState(int nodeId, Optional nodeDirectory, boolean hasAcknowledgedLeader) { - this.nodeId = nodeId; - this.nodeDirectory = nodeDirectory; + public ReplicaState(ReplicaKey replicaKey, boolean hasAcknowledgedLeader) { + this.replicaKey = replicaKey; this.endOffset = Optional.empty(); this.lastFetchTimestamp = -1; this.lastFetchLeaderLogEndOffset = -1; @@ -537,6 +555,41 @@ public class LeaderState implements EpochState { this.hasAcknowledgedLeader = hasAcknowledgedLeader; } + void setReplicaKey(ReplicaKey replicaKey) { + if (this.replicaKey.id() != replicaKey.id()) { + throw new IllegalArgumentException( + String.format( + "Attempting to update the replica key %s with a different replica id %s", + this.replicaKey, + replicaKey + ) + ); + } else if (this.replicaKey.directoryId().isPresent() && + !this.replicaKey.equals(replicaKey) + ) { + throw new IllegalArgumentException( + String.format( + "Attempting to update an already set directory id %s with a different directory id %s", + this.replicaKey, + replicaKey + ) + ); + } + + this.replicaKey = replicaKey; + } + + boolean matchesKey(ReplicaKey replicaKey) { + if (this.replicaKey.id() != replicaKey.id()) return false; + + if (this.replicaKey.directoryId().isPresent()) { + return this.replicaKey.directoryId().equals(replicaKey.directoryId()); + } else { + // it doesn't include a directory id so it matches as long as the ids match + return true; + } + } + void updateLeaderEndOffset( LogOffsetMetadata endOffsetMetadata ) { @@ -571,7 +624,7 @@ public class LeaderState implements EpochState { @Override public int compareTo(ReplicaState that) { if (this.endOffset.equals(that.endOffset)) - return Integer.compare(this.nodeId, that.nodeId); + return this.replicaKey.compareTo(that.replicaKey); else if (!this.endOffset.isPresent()) return 1; else if (!that.endOffset.isPresent()) @@ -583,10 +636,9 @@ public class LeaderState implements EpochState { @Override public String toString() { return String.format( - "ReplicaState(nodeId=%d, nodeDirectoryId=%s, endOffset=%s, lastFetchTimestamp=%s, " + - "lastCaughtUpTimestamp=%s, hasAcknowledgedLeader=%s)", - nodeId, - nodeDirectory, + "ReplicaState(replicaKey=%s, endOffset=%s, lastFetchTimestamp=%s, " + + "lastCaughtUpTimestamp=%s, hasAcknowledgedLeader=%s)", + replicaKey, endOffset, lastFetchTimestamp, lastCaughtUpTimestamp, @@ -608,9 +660,8 @@ public class LeaderState implements EpochState { @Override public String toString() { return String.format( - "Leader(localId=%d, localDirectoryId=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)", - localId, - localDirectoryId, + "Leader(localReplicaKey=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)", + localReplicaKey, epoch, epochStartOffset, highWatermark, @@ -627,5 +678,4 @@ public class LeaderState implements EpochState { public void close() { accumulator.close(); } - } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 6e8da1eac41..fc55ca747f7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -16,9 +16,7 @@ */ package org.apache.kafka.raft; -import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.internals.BatchAccumulator; @@ -84,9 +82,9 @@ public class QuorumState { private final Time time; private final Logger log; private final QuorumStateStore store; - private final ListenerName listenerName; private final Supplier latestVoterSet; private final Supplier latestKraftVersion; + private final Endpoints localListeners; private final Random random; private final int electionTimeoutMs; private final int fetchTimeoutMs; @@ -97,9 +95,9 @@ public class QuorumState { public QuorumState( OptionalInt localId, Uuid localDirectoryId, - ListenerName listenerName, Supplier latestVoterSet, Supplier latestKraftVersion, + Endpoints localListeners, int electionTimeoutMs, int fetchTimeoutMs, QuorumStateStore store, @@ -109,9 +107,9 @@ public class QuorumState { ) { this.localId = localId; this.localDirectoryId = localDirectoryId; - this.listenerName = listenerName; this.latestVoterSet = latestVoterSet; this.latestKraftVersion = latestKraftVersion; + this.localListeners = localListeners; this.electionTimeoutMs = electionTimeoutMs; this.fetchTimeoutMs = fetchTimeoutMs; this.store = store; @@ -174,11 +172,12 @@ public class QuorumState { latestVoterSet.get().voterIds(), randomElectionTimeoutMs(), Collections.emptyList(), + localListeners, logContext ); } else if ( localId.isPresent() && - election.isVotedCandidate(ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId))) + election.isVotedCandidate(ReplicaKey.of(localId.getAsInt(), localDirectoryId)) ) { initialState = new CandidateState( time, @@ -202,27 +201,13 @@ public class QuorumState { logContext ); } else if (election.hasLeader()) { - /* KAFKA-16529 is going to change this so that the leader is not required to be in the set - * of voters. In other words, don't throw an IllegalStateException if the leader is not in - * the set of voters. - */ - Node leader = latestVoterSet - .get() - .voterNode(election.leaderId(), listenerName) - .orElseThrow(() -> - new IllegalStateException( - String.format( - "Leader %s must be in the voter set %s", - election.leaderId(), - latestVoterSet.get() - ) - ) - ); + VoterSet voters = latestVoterSet.get(); initialState = new FollowerState( time, election.epoch(), - leader, - latestVoterSet.get().voterIds(), + election.leaderId(), + voters.listeners(election.leaderId()), + voters.voterIds(), Optional.empty(), fetchTimeoutMs, logContext @@ -243,9 +228,7 @@ public class QuorumState { public boolean isOnlyVoter() { return localId.isPresent() && - latestVoterSet.get().isOnlyVoter( - ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId)) - ); + latestVoterSet.get().isOnlyVoter(ReplicaKey.of(localId.getAsInt(), localDirectoryId)); } public int localIdOrSentinel() { @@ -277,7 +260,6 @@ public class QuorumState { } public OptionalInt leaderId() { - ElectionState election = state.election(); if (election.hasLeader()) return OptionalInt.of(state.election().leaderId()); @@ -293,6 +275,10 @@ public class QuorumState { return hasLeader() && leaderIdOrSentinel() != localIdOrSentinel(); } + public Endpoints leaderEndpoints() { + return state.leaderEndpoints(); + } + public boolean isVoter() { if (!localId.isPresent()) { return false; @@ -300,7 +286,7 @@ public class QuorumState { return latestVoterSet .get() - .isVoter(ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId))); + .isVoter(ReplicaKey.of(localId.getAsInt(), localDirectoryId)); } public boolean isVoter(ReplicaKey nodeKey) { @@ -327,6 +313,7 @@ public class QuorumState { latestVoterSet.get().voterIds(), randomElectionTimeoutMs(), preferredSuccessors, + localListeners, logContext ) ); @@ -427,16 +414,16 @@ public class QuorumState { /** * Become a follower of an elected leader so that we can begin fetching. */ - public void transitionToFollower(int epoch, Node leader) { + public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { int currentEpoch = state.epoch(); - if (localId.isPresent() && leader.id() == localId.getAsInt()) { - throw new IllegalStateException("Cannot transition to Follower with leader " + leader + + if (localId.isPresent() && leaderId == localId.getAsInt()) { + throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId + " and epoch " + epoch + " since it matches the local broker.id " + localId); } else if (epoch < currentEpoch) { - throw new IllegalStateException("Cannot transition to Follower with leader " + leader + + throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId + " and epoch " + epoch + " since the current epoch " + currentEpoch + " is larger"); } else if (epoch == currentEpoch && (isFollower() || isLeader())) { - throw new IllegalStateException("Cannot transition to Follower with leader " + leader + + throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId + " and epoch " + epoch + " from state " + state); } @@ -444,7 +431,8 @@ public class QuorumState { new FollowerState( time, epoch, - leader, + leaderId, + endpoints, latestVoterSet.get().voterIds(), state.highWatermark(), fetchTimeoutMs, @@ -518,13 +506,13 @@ public class QuorumState { LeaderState state = new LeaderState<>( time, - localIdOrThrow(), - localDirectoryId(), + ReplicaKey.of(localIdOrThrow(), localDirectoryId), epoch(), epochStartOffset, - latestVoterSet.get().voters(), + latestVoterSet.get(), candidateState.grantingVoters(), accumulator, + localListeners, fetchTimeoutMs, logContext ); diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftMessage.java b/raft/src/main/java/org/apache/kafka/raft/RaftMessage.java index a3e71ed1cc1..ad1e53df964 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftMessage.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftMessage.java @@ -21,7 +21,5 @@ import org.apache.kafka.common.protocol.ApiMessage; public interface RaftMessage { int correlationId(); - short apiVersion(); - ApiMessage data(); } diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java b/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java index a7ad8f2e61f..715cde5d720 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java @@ -17,19 +17,18 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Node; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiMessage; import java.util.concurrent.CompletableFuture; public abstract class RaftRequest implements RaftMessage { private final int correlationId; - private final short apiVersion; private final ApiMessage data; private final long createdTimeMs; - public RaftRequest(int correlationId, short apiVersion, ApiMessage data, long createdTimeMs) { + public RaftRequest(int correlationId, ApiMessage data, long createdTimeMs) { this.correlationId = correlationId; - this.apiVersion = apiVersion; this.data = data; this.createdTimeMs = createdTimeMs; } @@ -39,11 +38,6 @@ public abstract class RaftRequest implements RaftMessage { return correlationId; } - @Override - public short apiVersion() { - return apiVersion; - } - @Override public ApiMessage data() { return data; @@ -54,17 +48,40 @@ public abstract class RaftRequest implements RaftMessage { } public static final class Inbound extends RaftRequest { + private final short apiVersion; + private final ListenerName listenerName; + public final CompletableFuture completion = new CompletableFuture<>(); - public Inbound(int correlationId, short apiVersion, ApiMessage data, long createdTimeMs) { - super(correlationId, apiVersion, data, createdTimeMs); + public Inbound( + ListenerName listenerName, + int correlationId, + short apiVersion, + ApiMessage data, + long createdTimeMs + ) { + super(correlationId, data, createdTimeMs); + + this.listenerName = listenerName; + this.apiVersion = apiVersion; + } + + public short apiVersion() { + return apiVersion; + } + + public ListenerName listenerName() { + return listenerName; } @Override public String toString() { return String.format( - "InboundRequest(correlationId=%d, data=%s, createdTimeMs=%d)", + "InboundRequest(listenerName=%s, correlationId=%d, apiVersion=%d, data=%s, " + + "createdTimeMs=%d)", + listenerName, correlationId(), + apiVersion, data(), createdTimeMs() ); @@ -75,8 +92,8 @@ public abstract class RaftRequest implements RaftMessage { private final Node destination; public final CompletableFuture completion = new CompletableFuture<>(); - public Outbound(int correlationId, short apiVersion, ApiMessage data, Node destination, long createdTimeMs) { - super(correlationId, apiVersion, data, createdTimeMs); + public Outbound(int correlationId, ApiMessage data, Node destination, long createdTimeMs) { + super(correlationId, data, createdTimeMs); this.destination = destination; } diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java b/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java index bc7c0f4eff3..591b68f19f5 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java @@ -33,11 +33,6 @@ public abstract class RaftResponse implements RaftMessage { return correlationId; } - @Override - public short apiVersion() { - return data().highestSupportedVersion(); - } - @Override public ApiMessage data() { return data; diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java index 86a47eff1c8..c28864b2f2b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -21,22 +21,30 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.message.DescribeQuorumRequestData; +import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteResponseData; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.raft.internals.ReplicaKey; +import java.net.InetSocketAddress; import java.util.Collections; +import java.util.List; +import java.util.Optional; import java.util.function.Consumer; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; -import static java.util.Collections.singletonList; - +@SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity" }) public class RaftUtil { public static ApiMessage errorResponse(ApiKeys apiKey, Errors error) { @@ -70,16 +78,20 @@ public class RaftUtil { new FetchRequestData.FetchTopic() .setTopic(topicPartition.topic()) .setTopicId(topicId) - .setPartitions(singletonList(fetchPartition)); + .setPartitions(Collections.singletonList(fetchPartition)); return new FetchRequestData() - .setTopics(singletonList(fetchTopic)); + .setTopics(Collections.singletonList(fetchTopic)); } public static FetchResponseData singletonFetchResponse( + ListenerName listenerName, + short apiVersion, TopicPartition topicPartition, Uuid topicId, Errors topLevelError, + int leaderId, + Endpoints endpoints, Consumer partitionConsumer ) { FetchResponseData.PartitionData fetchablePartition = @@ -95,11 +107,421 @@ public class RaftUtil { .setTopicId(topicId) .setPartitions(Collections.singletonList(fetchablePartition)); - return new FetchResponseData() + FetchResponseData response = new FetchResponseData(); + + if (apiVersion >= 17) { + Optional address = endpoints.address(listenerName); + if (address.isPresent() && leaderId >= 0) { + // Populate the node endpoints + FetchResponseData.NodeEndpointCollection nodeEndpoints = new FetchResponseData.NodeEndpointCollection(1); + nodeEndpoints.add( + new FetchResponseData.NodeEndpoint() + .setNodeId(leaderId) + .setHost(address.get().getHostString()) + .setPort(address.get().getPort()) + ); + response.setNodeEndpoints(nodeEndpoints); + } + } + + return response .setErrorCode(topLevelError.code()) .setResponses(Collections.singletonList(fetchableTopic)); } + public static VoteRequestData singletonVoteRequest( + TopicPartition topicPartition, + String clusterId, + int candidateEpoch, + ReplicaKey candidateKey, + ReplicaKey voterKey, + int lastEpoch, + long lastEpochEndOffset + ) { + return new VoteRequestData() + .setClusterId(clusterId) + .setVoterId(voterKey.id()) + .setTopics( + Collections.singletonList( + new VoteRequestData.TopicData() + .setTopicName(topicPartition.topic()) + .setPartitions( + Collections.singletonList( + new VoteRequestData.PartitionData() + .setPartitionIndex(topicPartition.partition()) + .setCandidateEpoch(candidateEpoch) + .setCandidateId(candidateKey.id()) + .setCandidateDirectoryId( + candidateKey + .directoryId() + .orElse(ReplicaKey.NO_DIRECTORY_ID) + ) + .setVoterDirectoryId( + voterKey + .directoryId() + .orElse(ReplicaKey.NO_DIRECTORY_ID) + ) + .setLastOffsetEpoch(lastEpoch) + .setLastOffset(lastEpochEndOffset) + ) + ) + ) + ); + } + + public static VoteResponseData singletonVoteResponse( + ListenerName listenerName, + short apiVersion, + Errors topLevelError, + TopicPartition topicPartition, + Errors partitionLevelError, + int leaderEpoch, + int leaderId, + boolean voteGranted, + Endpoints endpoints + ) { + VoteResponseData response = new VoteResponseData() + .setErrorCode(topLevelError.code()) + .setTopics(Collections.singletonList( + new VoteResponseData.TopicData() + .setTopicName(topicPartition.topic()) + .setPartitions(Collections.singletonList( + new VoteResponseData.PartitionData() + .setErrorCode(partitionLevelError.code()) + .setLeaderId(leaderId) + .setLeaderEpoch(leaderEpoch) + .setVoteGranted(voteGranted))))); + + if (apiVersion >= 1) { + Optional 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 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 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 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 preferredReplicaKeys + ) { + List preferredSuccessors = preferredReplicaKeys + .stream() + .map(ReplicaKey::id) + .collect(Collectors.toList()); + + List 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 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 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 beginQuorumEpochRequestVoterKey( + BeginQuorumEpochRequestData request, + BeginQuorumEpochRequestData.PartitionData partition + ) { + if (request.voterId() < 0) { + return Optional.empty(); + } else { + return Optional.of(ReplicaKey.of(request.voterId(), partition.voterDirectoryId())); + } + } + static boolean hasValidTopicPartition(FetchRequestData data, TopicPartition topicPartition, Uuid topicId) { return data.topics().size() == 1 && data.topics().get(0).topicId().equals(topicId) && diff --git a/raft/src/main/java/org/apache/kafka/raft/RequestManager.java b/raft/src/main/java/org/apache/kafka/raft/RequestManager.java index 4e151958363..42f19eb40bc 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RequestManager.java +++ b/raft/src/main/java/org/apache/kafka/raft/RequestManager.java @@ -133,7 +133,7 @@ public class RequestManager { * If there is a connection with a pending request it returns the amount of time to wait until * the request times out. * - * Returns zero, if there are no pending request and at least one of the boorstrap servers is + * Returns zero, if there are no pending requests and at least one of the boorstrap servers is * ready. * * If all of the bootstrap servers are backing off and there are no pending requests, return diff --git a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java index a28c3155594..557c33cae13 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java @@ -44,6 +44,7 @@ import java.util.Set; public class ResignedState implements EpochState { private final int localId; private final int epoch; + private final Endpoints endpoints; private final Set voters; private final long electionTimeoutMs; private final Set unackedVoters; @@ -58,6 +59,7 @@ public class ResignedState implements EpochState { Set voters, long electionTimeoutMs, List preferredSuccessors, + Endpoints endpoints, LogContext logContext ) { this.localId = localId; @@ -68,6 +70,7 @@ public class ResignedState implements EpochState { this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); this.preferredSuccessors = preferredSuccessors; + this.endpoints = endpoints; this.log = logContext.logger(ResignedState.class); } @@ -81,6 +84,11 @@ public class ResignedState implements EpochState { return epoch; } + @Override + public Endpoints leaderEndpoints() { + return endpoints; + } + /** * Get the set of voters which have yet to acknowledge the resignation. * This node will send `EndQuorumEpoch` requests to this set until these diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index e7805ce061e..3bfaeb41f2c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -65,6 +65,11 @@ public class UnattachedState implements EpochState { return epoch; } + @Override + public Endpoints leaderEndpoints() { + return Endpoints.empty(); + } + @Override public String name() { return "Unattached"; diff --git a/raft/src/main/java/org/apache/kafka/raft/VotedState.java b/raft/src/main/java/org/apache/kafka/raft/VotedState.java index b95803ce867..c7710903399 100644 --- a/raft/src/main/java/org/apache/kafka/raft/VotedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/VotedState.java @@ -74,6 +74,11 @@ public class VotedState implements EpochState { return epoch; } + @Override + public Endpoints leaderEndpoints() { + return Endpoints.empty(); + } + @Override public String name() { return "Voted"; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BlockingMessageQueue.java b/raft/src/main/java/org/apache/kafka/raft/internals/BlockingMessageQueue.java index 216a67640c9..9343cca8d47 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BlockingMessageQueue.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BlockingMessageQueue.java @@ -33,11 +33,6 @@ public class BlockingMessageQueue implements RaftMessageQueue { return 0; } - @Override - public short apiVersion() { - return 0; - } - @Override public ApiMessage data() { return null; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java b/raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java index a222787137a..4e7cea17c6b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java @@ -21,7 +21,9 @@ import org.apache.kafka.common.Uuid; import java.util.Objects; import java.util.Optional; -public final class ReplicaKey { +public final class ReplicaKey implements Comparable { + public static final Uuid NO_DIRECTORY_ID = Uuid.ZERO_UUID; + private final int id; private final Optional directoryId; @@ -38,6 +40,18 @@ public final class ReplicaKey { return directoryId; } + @Override + public int compareTo(ReplicaKey that) { + int idComparison = Integer.compare(this.id, that.id); + if (idComparison == 0) { + return directoryId + .orElse(NO_DIRECTORY_ID) + .compareTo(that.directoryId.orElse(NO_DIRECTORY_ID)); + } else { + return idComparison; + } + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -59,7 +73,10 @@ public final class ReplicaKey { return String.format("ReplicaKey(id=%d, directoryId=%s)", id, directoryId); } - public static ReplicaKey of(int id, Optional directoryId) { - return new ReplicaKey(id, directoryId); + public static ReplicaKey of(int id, Uuid directoryId) { + return new ReplicaKey( + id, + directoryId.equals(NO_DIRECTORY_ID) ? Optional.empty() : Optional.of(directoryId) + ); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index faff794c81d..2f961897982 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.raft.Endpoints; import java.net.InetSocketAddress; import java.util.Collections; @@ -97,28 +98,19 @@ public final class VoterSet { /** * Returns if the node is a voter in the set of voters. * - * If the voter set includes the directory id, the {@code nodeKey} directory id must match the + * If the voter set includes the directory id, the {@code replicaKey} directory id must match the * directory id specified by the voter set. * * If the voter set doesn't include the directory id ({@code Optional.empty()}), a node is in * the voter set as long as the node id matches. The directory id is not checked. * - * @param nodeKey the node's id and directory id + * @param replicaKey the node's id and directory id * @return true if the node is a voter in the voter set, otherwise false */ - public boolean isVoter(ReplicaKey nodeKey) { - VoterNode node = voters.get(nodeKey.id()); - if (node != null) { - if (node.voterKey().directoryId().isPresent()) { - return node.voterKey().directoryId().equals(nodeKey.directoryId()); - } else { - // configured voter set doesn't include a directory id so it is a voter as long as the node id - // matches - return true; - } - } else { - return false; - } + public boolean isVoter(ReplicaKey replicaKey) { + return Optional.ofNullable(voters.get(replicaKey.id())) + .map(node -> node.isVoter(replicaKey)) + .orElse(false); } /** @@ -138,8 +130,39 @@ public final class VoterSet { return voters.keySet(); } - public Map voters() { - return voters; + /** + * Returns all of the voters. + */ + public Set voterKeys() { + return voters + .values() + .stream() + .map(VoterNode::voterKey) + .collect(Collectors.toSet()); + } + + /** + * Returns all of the voters. + */ + public Set 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 voterConvertor = voter -> { Iterator endpoints = voter .listeners() - .entrySet() - .stream() - .map(entry -> - new VotersRecord.Endpoint() - .setName(entry.getKey().value()) - .setHost(entry.getValue().getHostString()) - .setPort(entry.getValue().getPort()) - ) - .iterator(); + .votersRecordEndpoints(); VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature() .setMinSupportedVersion(voter.supportedKRaftVersion().min()) @@ -239,17 +254,8 @@ public final class VoterSet { * @return true if they have an overlapping majority, false otherwise */ public boolean hasOverlappingMajority(VoterSet that) { - Set thisReplicaKeys = voters - .values() - .stream() - .map(VoterNode::voterKey) - .collect(Collectors.toSet()); - - Set thatReplicaKeys = that.voters - .values() - .stream() - .map(VoterNode::voterKey) - .collect(Collectors.toSet()); + Set thisReplicaKeys = voterKeys(); + Set thatReplicaKeys = that.voterKeys(); if (Utils.diff(HashSet::new, thisReplicaKeys, thatReplicaKeys).size() > 1) return false; return Utils.diff(HashSet::new, thatReplicaKeys, thisReplicaKeys).size() <= 1; @@ -277,12 +283,12 @@ public final class VoterSet { public static final class VoterNode { private final ReplicaKey voterKey; - private final Map listeners; + private final Endpoints listeners; private final SupportedVersionRange supportedKRaftVersion; - public VoterNode( + VoterNode( ReplicaKey voterKey, - Map listeners, + Endpoints listeners, SupportedVersionRange supportedKRaftVersion ) { this.voterKey = voterKey; @@ -294,7 +300,31 @@ public final class VoterSet { return voterKey; } - Map listeners() { + /** + * Returns if the provided replica key matches this voter node. + * + * If the voter node includes the directory id, the {@code replicaKey} directory id must + * match the directory id specified by the voter set. + * + * If the voter node doesn't include the directory id ({@code Optional.empty()}), a replica + * is the voter as long as the node id matches. The directory id is not checked. + * + * @param replicaKey the replica key + * @return true if the replica key is the voter, otherwise false + */ + public boolean isVoter(ReplicaKey replicaKey) { + if (voterKey.id() != replicaKey.id()) return false; + + if (voterKey.directoryId().isPresent()) { + return voterKey.directoryId().equals(replicaKey.directoryId()); + } else { + // configured voter set doesn't include a directory id so it is a voter as long as + // the ids match + return true; + } + } + + Endpoints listeners() { return listeners; } @@ -304,7 +334,7 @@ public final class VoterSet { Optional address(ListenerName listener) { - return Optional.ofNullable(listeners.get(listener)); + return listeners.address(listener); } @Override @@ -344,26 +374,11 @@ public final class VoterSet { public static VoterSet fromVotersRecord(VotersRecord voters) { HashMap voterNodes = new HashMap<>(voters.voters().size()); for (VotersRecord.Voter voter: voters.voters()) { - final Optional directoryId; - if (!voter.voterDirectoryId().equals(Uuid.ZERO_UUID)) { - directoryId = Optional.of(voter.voterDirectoryId()); - } else { - directoryId = Optional.empty(); - } - - Map listeners = new HashMap<>(voter.endpoints().size()); - for (VotersRecord.Endpoint endpoint : voter.endpoints()) { - listeners.put( - ListenerName.normalised(endpoint.name()), - InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port()) - ); - } - voterNodes.put( voter.voterId(), new VoterNode( - ReplicaKey.of(voter.voterId(), directoryId), - listeners, + ReplicaKey.of(voter.voterId(), voter.voterDirectoryId()), + Endpoints.fromVotersRecordEndpoints(voter.endpoints()), new SupportedVersionRange( voter.kRaftVersionFeature().minSupportedVersion(), voter.kRaftVersionFeature().maxSupportedVersion() @@ -390,8 +405,8 @@ public final class VoterSet { Collectors.toMap( Map.Entry::getKey, entry -> new VoterNode( - ReplicaKey.of(entry.getKey(), Optional.empty()), - Collections.singletonMap(listener, entry.getValue()), + ReplicaKey.of(entry.getKey(), Uuid.ZERO_UUID), + Endpoints.fromInetSocketAddresses(Collections.singletonMap(listener, entry.getValue())), new SupportedVersionRange((short) 0, (short) 0) ) ) diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index 42904f994d2..673b0fac133 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -204,10 +204,10 @@ public class CandidateStateTest { voterSetWithLocal(IntStream.of(1, 2, 3)) ); - assertFalse(state.canGrantVote(ReplicaKey.of(0, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(0, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); } @Test @@ -232,6 +232,15 @@ public class CandidateStateTest { ); } + @Test + void testLeaderEndpoints() { + CandidateState state = newCandidateState( + voterSetWithLocal(IntStream.of(1, 2, 3)) + ); + + assertEquals(Endpoints.empty(), state.leaderEndpoints()); + } + private VoterSet voterSetWithLocal(IntStream remoteVoters) { Map voterMap = VoterSetTest.voterMap(remoteVoters, true); voterMap.put(localNode.voterKey().id(), localNode); diff --git a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java index 26ec18bbe32..18b748061cb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java @@ -28,7 +28,6 @@ import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -38,38 +37,38 @@ final class ElectionStateTest { @Test void testVotedCandidateWithoutVotedId() { ElectionState electionState = ElectionState.withUnknownLeader(5, Collections.emptySet()); - assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty()))); + assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); } @Test void testVotedCandidateWithoutVotedDirectoryId() { ElectionState electionState = ElectionState.withVotedCandidate( 5, - ReplicaKey.of(1, Optional.empty()), + ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), Collections.emptySet() ); - assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty()))); + assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); assertTrue( - electionState.isVotedCandidate(ReplicaKey.of(1, Optional.of(Uuid.randomUuid()))) + electionState.isVotedCandidate(ReplicaKey.of(1, Uuid.randomUuid())) ); } @Test void testVotedCandidateWithVotedDirectoryId() { - ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid()); ElectionState electionState = ElectionState.withVotedCandidate( 5, votedKey, Collections.emptySet() ); - assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty()))); + assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); assertTrue(electionState.isVotedCandidate(votedKey)); } @ParameterizedTest @ValueSource(shorts = {0, 1}) void testQuorumStateDataRoundTrip(short version) { - ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid()); List electionStates = Arrays.asList( ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)), ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)), @@ -83,7 +82,7 @@ final class ElectionStateTest { ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)), ElectionState.withVotedCandidate( 5, - ReplicaKey.of(1, Optional.empty()), + ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), Utils.mkSet(1, 2, 3) ) ); diff --git a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java index b39f069ade4..15146a932a3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java @@ -84,8 +84,7 @@ public class FileQuorumStateStoreTest { final int epoch = 2; final int voter1 = 1; - final Optional voter1DirectoryId = Optional.of(Uuid.randomUuid()); - final ReplicaKey voter1Key = ReplicaKey.of(voter1, voter1DirectoryId); + final ReplicaKey voter1Key = ReplicaKey.of(voter1, Uuid.randomUuid()); final int voter2 = 2; final int voter3 = 3; Set voters = Utils.mkSet(voter1, voter2, voter3); @@ -108,7 +107,7 @@ public class FileQuorumStateStoreTest { expected = Optional.of( ElectionState.withVotedCandidate( epoch, - ReplicaKey.of(voter1, Optional.empty()), + ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID), voters ) ); diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index a6008905f07..01b20a0570b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.raft; -import org.apache.kafka.common.Node; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -26,6 +26,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.net.InetSocketAddress; +import java.util.Collections; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -40,7 +42,13 @@ public class FollowerStateTest { private final LogContext logContext = new LogContext(); private final int epoch = 5; private final int fetchTimeoutMs = 15000; - private final Node leader = new Node(3, "mock-host-3", 1234); + private final int leaderId = 3; + private final Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + ListenerName.normalised("CONTROLLER"), + InetSocketAddress.createUnresolved("mock-host-3", 1234) + ) + ); private FollowerState newFollowerState( Set voters, @@ -49,7 +57,8 @@ public class FollowerStateTest { return new FollowerState( time, epoch, - leader, + leaderId, + leaderEndpoints, voters, highWatermark, fetchTimeoutMs, @@ -93,15 +102,16 @@ public class FollowerStateTest { Optional.empty() ); - assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); } @Test - public void testLeaderNode() { + public void testLeaderIdAndEndpoint() { FollowerState state = newFollowerState(Utils.mkSet(0, 1, 2), Optional.empty()); - assertEquals(leader, state.leader()); + assertEquals(leaderId, state.leaderId()); + assertEquals(leaderEndpoints, state.leaderEndpoints()); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index 725c4ea1024..586d2ad54ee 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -233,7 +233,6 @@ public class KafkaNetworkChannelTest { ApiMessage apiRequest = buildTestRequest(apiKey); RaftRequest.Outbound request = new RaftRequest.Outbound( correlationId, - apiRequest.highestSupportedVersion(), apiRequest, destination, createdTimeMs @@ -322,7 +321,23 @@ public class KafkaNetworkChannelTest { case END_QUORUM_EPOCH: return new EndQuorumEpochResponseData().setErrorCode(error.code()); case VOTE: - return VoteResponse.singletonResponse(error, topicPartition, Errors.NONE, 1, 5, false); + return new VoteResponseData() + .setErrorCode(error.code()) + .setTopics( + Collections.singletonList( + new VoteResponseData.TopicData() + .setTopicName(topicPartition.topic()) + .setPartitions( + Collections.singletonList( + new VoteResponseData.PartitionData() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(1) + .setLeaderEpoch(5) + .setVoteGranted(false) + ) + ) + ) + ); case FETCH: return new FetchResponseData().setErrorCode(error.code()); case FETCH_SNAPSHOT: diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 3f8bc98559a..c6972f7302d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -25,8 +25,8 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.requests.FetchSnapshotRequest; -import org.apache.kafka.common.requests.FetchSnapshotResponse; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.StringSerde; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; @@ -37,6 +37,7 @@ import org.apache.kafka.snapshot.SnapshotWriterReaderTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; @@ -93,14 +94,15 @@ public final class KafkaRaftClientSnapshotTest { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testLeaderListenerNotified(boolean entireLog) throws Exception { + @CsvSource({"false,false", "false,true", "true,false", "true,true"}) + public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, false); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .withEmptySnapshot(snapshotId); @@ -116,7 +118,7 @@ public final class KafkaRaftClientSnapshotTest { // Advance the highWatermark long localLogEndOffset = context.log.endOffset().offset; - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, localLogEndOffset, epoch, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); @@ -216,14 +218,16 @@ public final class KafkaRaftClientSnapshotTest { } } - @Test - public void testListenerRenotified() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testListenerRenotified(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i")) @@ -239,7 +243,7 @@ public final class KafkaRaftClientSnapshotTest { // Advance the highWatermark long localLogEndOffset = context.log.endOffset().offset; - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, localLogEndOffset, epoch, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); @@ -270,15 +274,17 @@ public final class KafkaRaftClientSnapshotTest { } } - @Test - public void testLeaderImmediatelySendsSnapshotId() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(snapshotId.epoch()) + .withKip853Rpc(withKip853Rpc) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i")) @@ -290,7 +296,7 @@ public final class KafkaRaftClientSnapshotTest { int epoch = context.currentEpoch(); // Send a fetch request for an end offset and epoch which has been snapshotted - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 6, 2, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 6, 2, 500)); context.client.poll(); // Expect that the leader replies immediately with a snapshot id @@ -302,14 +308,16 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset()); } - @Test - public void testFetchRequestOffsetLessThanLogStart() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -338,7 +346,7 @@ public final class KafkaRaftClientSnapshotTest { context.client.poll(); // Send Fetch request less than start offset - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, snapshotId.offset() - 2, snapshotId.epoch(), 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, snapshotId.offset() - 2, snapshotId.epoch(), 0)); context.pollUntilResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); @@ -348,15 +356,17 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset()); } - @Test - public void testFetchRequestOffsetAtZero() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception { // When the follower sends a FETCH request at offset 0, reply with snapshot id if it exists int localId = 0; - int otherNodeId = localId + 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -384,7 +394,7 @@ public final class KafkaRaftClientSnapshotTest { } // Send Fetch request for offset 0 - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0, 0, 0)); context.pollUntilResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); @@ -394,11 +404,12 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset()); } - @Test - public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); @@ -406,6 +417,7 @@ public final class KafkaRaftClientSnapshotTest { .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -427,17 +439,18 @@ public final class KafkaRaftClientSnapshotTest { context.client.poll(); // It is an invalid request to send an last fetched epoch greater than the current epoch - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset() + 1, epoch + 1, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, oldestSnapshotId.offset() + 1, epoch + 1, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); } - @Test - public void testFetchRequestTruncateToLogStart() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - int syncNodeId = otherNodeId + 1; - Set voters = Utils.mkSet(localId, otherNodeId, syncNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + int syncNodeId = otherNodeKey.id() + 1; + Set voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); @@ -445,6 +458,7 @@ public final class KafkaRaftClientSnapshotTest { .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("d", "e", "f")) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -463,7 +477,13 @@ public final class KafkaRaftClientSnapshotTest { // This should truncate to the old snapshot context.deliverRequest( - context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset() + 1, oldestSnapshotId.epoch() + 1, 0) + context.fetchRequest( + epoch, + otherNodeKey, + oldestSnapshotId.offset() + 1, + oldestSnapshotId.epoch() + 1, + 0 + ) ); context.pollUntilResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); @@ -474,12 +494,13 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(oldestSnapshotId.offset(), partitionResponse.divergingEpoch().endOffset()); } - @Test - public void testFetchRequestAtLogStartOffsetWithValidEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - int syncNodeId = otherNodeId + 1; - Set voters = Utils.mkSet(localId, otherNodeId, syncNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + int syncNodeId = otherNodeKey.id() + 1; + Set voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); @@ -488,6 +509,7 @@ public final class KafkaRaftClientSnapshotTest { .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -506,18 +528,25 @@ public final class KafkaRaftClientSnapshotTest { // Send fetch request at log start offset with valid last fetched epoch context.deliverRequest( - context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch(), 0) + context.fetchRequest( + epoch, + otherNodeKey, + oldestSnapshotId.offset(), + oldestSnapshotId.epoch(), + 0 + ) ); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); } - @Test - public void testFetchRequestAtLogStartOffsetWithInvalidEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - int syncNodeId = otherNodeId + 1; - Set voters = Utils.mkSet(localId, otherNodeId, syncNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + int syncNodeId = otherNodeKey.id() + 1; + Set voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); @@ -526,6 +555,7 @@ public final class KafkaRaftClientSnapshotTest { .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -545,7 +575,13 @@ public final class KafkaRaftClientSnapshotTest { // Send fetch with log start offset and invalid last fetched epoch context.deliverRequest( - context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch() + 1, 0) + context.fetchRequest( + epoch, + otherNodeKey, + oldestSnapshotId.offset(), + oldestSnapshotId.epoch() + 1, + 0 + ) ); context.pollUntilResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); @@ -556,12 +592,15 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset()); } - @Test - public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( + boolean withKip853Rpc + ) throws Exception { int localId = 0; - int otherNodeId = localId + 1; - int syncNodeId = otherNodeId + 1; - Set voters = Utils.mkSet(localId, otherNodeId, syncNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + int syncNodeId = otherNodeKey.id() + 1; + Set voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId); OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); @@ -570,6 +609,7 @@ public final class KafkaRaftClientSnapshotTest { .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .withAppendLingerMs(1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -590,7 +630,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverRequest( context.fetchRequest( epoch, - otherNodeId, + otherNodeKey, context.log.endOffset().offset, oldestSnapshotId.epoch() - 1, 0 @@ -605,13 +645,19 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset()); } - @Test - public void testFetchSnapshotRequestMissingSnapshot() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throws Exception { int localId = 0; - int epoch = 2; Set voters = Utils.mkSet(localId, localId + 1); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(3) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); context.deliverRequest( fetchSnapshotRequest( @@ -629,14 +675,20 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(Errors.SNAPSHOT_NOT_FOUND, Errors.forCode(response.errorCode())); } - @Test - public void testFetchSnapshotRequestUnknownPartition() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Utils.mkSet(localId, localId + 1); - int epoch = 2; TopicPartition topicPartition = new TopicPartition("unknown", 0); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(3) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); context.deliverRequest( fetchSnapshotRequest( @@ -654,8 +706,9 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.forCode(response.errorCode())); } - @Test - public void testFetchSnapshotRequestAsLeader() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Utils.mkSet(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); @@ -663,6 +716,7 @@ public final class KafkaRaftClientSnapshotTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), Collections.singletonList("a")) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -703,18 +757,22 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(memoryRecords.buffer(), ((UnalignedMemoryRecords) response.unalignedRecords()).buffer()); } - @Test - public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters( + boolean withKip853Rpc + ) throws Exception { int localId = 0; - int voter1 = 1; - int voter2 = 2; - int observerId3 = 3; - Set voters = Utils.mkSet(localId, voter1, voter2); + ReplicaKey voter1 = replicaKey(1, withKip853Rpc); + ReplicaKey voter2 = replicaKey(2, withKip853Rpc); + ReplicaKey observer3 = replicaKey(3, withKip853Rpc); + Set voters = Utils.mkSet(localId, voter1.id(), voter2.id()); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); List records = Arrays.asList("foo", "bar"); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), Collections.singletonList("a")) + .withKip853Rpc(withKip853Rpc) .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; @@ -743,7 +801,7 @@ public final class KafkaRaftClientSnapshotTest { FetchSnapshotRequestData observerFetchSnapshotRequest = fetchSnapshotRequest( context.clusterId.toString(), - observerId3, + observer3, context.metadataPartition, epoch, snapshotId, @@ -794,8 +852,9 @@ public final class KafkaRaftClientSnapshotTest { assertTrue(context.client.quorum().isResigned()); } - @Test - public void testPartialFetchSnapshotRequestAsLeader() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Utils.mkSet(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1); @@ -803,6 +862,7 @@ public final class KafkaRaftClientSnapshotTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), records) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -873,8 +933,9 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(snapshotBuffer, responseBuffer.flip()); } - @Test - public void testFetchSnapshotRequestAsFollower() throws IOException { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOException { int localId = 0; int leaderId = localId + 1; Set voters = Utils.mkSet(localId, leaderId); @@ -883,6 +944,7 @@ public final class KafkaRaftClientSnapshotTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leaderId) + .withKip853Rpc(withKip853Rpc) .build(); context.deliverRequest( @@ -903,8 +965,9 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(leaderId, response.currentLeader().leaderId()); } - @Test - public void testFetchSnapshotRequestWithInvalidPosition() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Utils.mkSet(localId, localId + 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); @@ -912,6 +975,7 @@ public final class KafkaRaftClientSnapshotTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(snapshotId.epoch(), Collections.singletonList("a")) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -961,14 +1025,20 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(localId, response.currentLeader().leaderId()); } - @Test - public void testFetchSnapshotRequestWithOlderEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Utils.mkSet(localId, localId + 1); - int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(0, 0); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(1) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); context.deliverRequest( fetchSnapshotRequest( @@ -988,14 +1058,20 @@ public final class KafkaRaftClientSnapshotTest { assertEquals(localId, response.currentLeader().leaderId()); } - @Test - public void testFetchSnapshotRequestWithNewerEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Utils.mkSet(localId, localId + 1); - int epoch = 2; OffsetAndEpoch snapshotId = new OffsetAndEpoch(0, 0); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(1) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); context.deliverRequest( fetchSnapshotRequest( @@ -1036,7 +1112,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEpoch, 200L) + context.snapshotFetchResponse(epoch, leaderId, invalidEpoch, 200L) ); // Handle the invalid response @@ -1053,7 +1129,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEndOffset, 200L) + context.snapshotFetchResponse(epoch, leaderId, invalidEndOffset, 200L) ); // Handle the invalid response @@ -1095,7 +1171,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1121,7 +1197,7 @@ public final class KafkaRaftClientSnapshotTest { snapshotRequest.correlationId(), snapshotRequest.destination(), fetchSnapshotResponse( - context.metadataPartition, + context, epoch, leaderId, snapshotId, @@ -1166,7 +1242,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1195,7 +1271,7 @@ public final class KafkaRaftClientSnapshotTest { snapshotRequest.correlationId(), snapshotRequest.destination(), fetchSnapshotResponse( - context.metadataPartition, + context, epoch, leaderId, snapshotId, @@ -1224,7 +1300,7 @@ public final class KafkaRaftClientSnapshotTest { snapshotRequest.correlationId(), snapshotRequest.destination(), fetchSnapshotResponse( - context.metadataPartition, + context, epoch, leaderId, snapshotId, @@ -1269,7 +1345,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1288,8 +1364,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + leaderId, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1327,7 +1403,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, firstLeaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, firstLeaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1346,8 +1422,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + secondLeaderId, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1384,7 +1460,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1403,8 +1479,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + leaderId, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1441,7 +1517,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1460,8 +1536,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + leaderId + 1, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1508,7 +1584,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1527,8 +1603,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + leaderId, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1554,7 +1630,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1574,8 +1650,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + leaderId, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1618,7 +1694,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) + context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L) ); context.pollUntilRequest(); @@ -1646,8 +1722,8 @@ public final class KafkaRaftClientSnapshotTest { context.deliverResponse( snapshotRequest.correlationId(), snapshotRequest.destination(), - FetchSnapshotResponse.singleton( - context.metadataPartition, + context.fetchSnapshotResponse( + leaderId, responsePartitionSnapshot -> { responsePartitionSnapshot .currentLeader() @@ -1669,20 +1745,28 @@ public final class KafkaRaftClientSnapshotTest { context.assertVotedCandidate(epoch + 1, localId); } - @Test - public void testFetchSnapshotRequestClusterIdValidation() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFetchSnapshotRequestClusterIdValidation( + boolean withKip853Rpc + ) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNode = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNode.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); - // null cluster id is accepted + context.becomeLeader(); + int epoch = context.currentEpoch(); + + // valid cluster id is accepted context.deliverRequest( fetchSnapshotRequest( context.clusterId.toString(), - otherNodeId, + otherNode, context.metadataPartition, epoch, new OffsetAndEpoch(0, 0), @@ -1697,7 +1781,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverRequest( fetchSnapshotRequest( null, - otherNodeId, + otherNode, context.metadataPartition, epoch, new OffsetAndEpoch(0, 0), @@ -1712,7 +1796,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverRequest( fetchSnapshotRequest( "", - otherNodeId, + otherNode, context.metadataPartition, epoch, new OffsetAndEpoch(0, 0), @@ -1727,7 +1811,7 @@ public final class KafkaRaftClientSnapshotTest { context.deliverRequest( fetchSnapshotRequest( "invalid-uuid", - otherNodeId, + otherNode, context.metadataPartition, epoch, new OffsetAndEpoch(0, 0), @@ -1852,6 +1936,11 @@ public final class KafkaRaftClientSnapshotTest { assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4, 0)); } + private static ReplicaKey replicaKey(int id, boolean withDirectoryId) { + Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; + return ReplicaKey.of(id, directoryId); + } + private static FetchSnapshotRequestData fetchSnapshotRequest( TopicPartition topicPartition, int epoch, @@ -1859,37 +1948,39 @@ public final class KafkaRaftClientSnapshotTest { int maxBytes, long position ) { - return fetchSnapshotRequest(null, -1, topicPartition, epoch, offsetAndEpoch, maxBytes, position); + return RaftUtil.singletonFetchSnapshotRequest( + null, + ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID), + topicPartition, + epoch, + offsetAndEpoch, + maxBytes, + position + ); } private static FetchSnapshotRequestData fetchSnapshotRequest( String clusterId, - int replicaId, + ReplicaKey replicaKey, TopicPartition topicPartition, int epoch, OffsetAndEpoch offsetAndEpoch, int maxBytes, long position ) { - FetchSnapshotRequestData.SnapshotId snapshotId = new FetchSnapshotRequestData.SnapshotId() - .setEndOffset(offsetAndEpoch.offset()) - .setEpoch(offsetAndEpoch.epoch()); - - FetchSnapshotRequestData request = FetchSnapshotRequest.singleton( + return RaftUtil.singletonFetchSnapshotRequest( clusterId, - replicaId, + replicaKey, topicPartition, - snapshotPartition -> snapshotPartition - .setCurrentLeaderEpoch(epoch) - .setSnapshotId(snapshotId) - .setPosition(position) + epoch, + offsetAndEpoch, + maxBytes, + position ); - - return request.setMaxBytes(maxBytes); } private static FetchSnapshotResponseData fetchSnapshotResponse( - TopicPartition topicPartition, + RaftClientTestContext context, int leaderEpoch, int leaderId, OffsetAndEpoch snapshotId, @@ -1897,8 +1988,8 @@ public final class KafkaRaftClientSnapshotTest { long position, ByteBuffer buffer ) { - return FetchSnapshotResponse.singleton( - topicPartition, + return context.fetchSnapshotResponse( + leaderId, partitionSnapshot -> { partitionSnapshot.currentLeader() .setLeaderEpoch(leaderEpoch) @@ -1916,27 +2007,6 @@ public final class KafkaRaftClientSnapshotTest { ); } - private static FetchResponseData snapshotFetchResponse( - TopicPartition topicPartition, - Uuid topicId, - int epoch, - int leaderId, - OffsetAndEpoch snapshotId, - long highWatermark - ) { - return RaftUtil.singletonFetchResponse(topicPartition, topicId, Errors.NONE, partitionData -> { - partitionData.setHighWatermark(highWatermark); - - partitionData.currentLeader() - .setLeaderEpoch(epoch) - .setLeaderId(leaderId); - - partitionData.snapshotId() - .setEpoch(snapshotId.epoch()) - .setEndOffset(snapshotId.offset()); - }); - } - private static Optional assertFetchSnapshotRequest( RaftRequest.Outbound request, TopicPartition topicPartition, diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 23b3447543b..c1a49a2b788 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -36,8 +36,6 @@ import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.Records; -import org.apache.kafka.common.requests.DescribeQuorumRequest; -import org.apache.kafka.common.requests.EndQuorumEpochResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; @@ -49,6 +47,7 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; @@ -107,16 +106,18 @@ public class KafkaRaftClientTest { context.assertElectedLeader(initialEpoch + 1, localId); } - @Test - public void testRejectVotesFromSameEpochAfterResigningLeadership() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testRejectVotesFromSameEpochAfterResigningLeadership(boolean withKip853Rpc) throws Exception { int localId = 0; - int remoteId = 1; - Set voters = Utils.mkSet(localId, remoteId); + ReplicaKey remoteKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withElectedLeader(epoch, localId) + .withKip853Rpc(withKip853Rpc) .build(); assertEquals(0L, context.log.endOffset().offset); @@ -124,22 +125,30 @@ public class KafkaRaftClientTest { // Since we were the leader in epoch 2, we should ensure that we will not vote for any // other voter in the same epoch, even if it has caught up to the same position. - context.deliverRequest(context.voteRequest(epoch, remoteId, - context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.deliverRequest( + context.voteRequest( + epoch, + remoteKey, + context.log.lastFetchedEpoch(), + context.log.endOffset().offset + ) + ); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), false); } - @Test - public void testRejectVotesFromSameEpochAfterResigningCandidacy() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testRejectVotesFromSameEpochAfterResigningCandidacy(boolean withKip853Rpc) throws Exception { int localId = 0; - int remoteId = 1; - Set voters = Utils.mkSet(localId, remoteId); + ReplicaKey remoteKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) - .withVotedCandidate(epoch, ReplicaKey.of(localId, Optional.empty())) + .withVotedCandidate(epoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + .withKip853Rpc(withKip853Rpc) .build(); assertEquals(0L, context.log.endOffset().offset); @@ -147,22 +156,30 @@ public class KafkaRaftClientTest { // Since we were the leader in epoch 2, we should ensure that we will not vote for any // other voter in the same epoch, even if it has caught up to the same position. - context.deliverRequest(context.voteRequest(epoch, remoteId, - context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.deliverRequest( + context.voteRequest( + epoch, + remoteKey, + context.log.lastFetchedEpoch(), + context.log.endOffset().offset + ) + ); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), false); } - @Test - public void testGrantVotesFromHigherEpochAfterResigningLeadership() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKip853Rpc) throws Exception { int localId = 0; - int remoteId = 1; - Set voters = Utils.mkSet(localId, remoteId); + ReplicaKey remoteKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withElectedLeader(epoch, localId) + .withKip853Rpc(withKip853Rpc) .build(); // Resign from leader, will restart in resigned state @@ -171,26 +188,34 @@ public class KafkaRaftClientTest { context.assertElectedLeader(epoch, localId); // Send vote request with higher epoch - context.deliverRequest(context.voteRequest(epoch + 1, remoteId, - context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.deliverRequest( + context.voteRequest( + epoch + 1, + remoteKey, + context.log.lastFetchedEpoch(), + context.log.endOffset().offset + ) + ); context.client.poll(); // We will first transition to unattached and then grant vote and then transition to voted assertTrue(context.client.quorum().isVoted()); - context.assertVotedCandidate(epoch + 1, remoteId); + context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); } - @Test - public void testGrantVotesFromHigherEpochAfterResigningCandidacy() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip853Rpc) throws Exception { int localId = 0; - int remoteId = 1; - Set voters = Utils.mkSet(localId, remoteId); + ReplicaKey remoteKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, remoteKey.id()); int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) - .withVotedCandidate(epoch, ReplicaKey.of(localId, Optional.empty())) + .withVotedCandidate(epoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + .withKip853Rpc(withKip853Rpc) .build(); // Resign from candidate, will restart in candidate state @@ -199,37 +224,54 @@ public class KafkaRaftClientTest { context.assertVotedCandidate(epoch, localId); // Send vote request with higher epoch - context.deliverRequest(context.voteRequest(epoch + 1, remoteId, - context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.deliverRequest( + context.voteRequest( + epoch + 1, + remoteKey, + context.log.lastFetchedEpoch(), + context.log.endOffset().offset + ) + ); context.client.poll(); // We will first transition to unattached and then grant vote and then transition to voted assertTrue(context.client.quorum().isVoted()); - context.assertVotedCandidate(epoch + 1, remoteId); + context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); } - @Test - public void testGrantVotesWhenShuttingDown() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Exception { int localId = 0; - int remoteId = 1; - Set voters = Utils.mkSet(localId, remoteId); - int epoch = 2; + ReplicaKey remoteKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, remoteKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // Beginning shutdown context.client.shutdown(1000); assertTrue(context.client.isShuttingDown()); // Send vote request with higher epoch - context.deliverRequest(context.voteRequest(epoch + 1, remoteId, - context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.deliverRequest( + context.voteRequest( + epoch + 1, + remoteKey, + context.log.lastFetchedEpoch(), + context.log.endOffset().offset + ) + ); context.client.poll(); // We will first transition to unattached and then grant vote and then transition to voted assertTrue(context.client.quorum().isVoted()); - context.assertVotedCandidate(epoch + 1, remoteId); + context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); } @@ -415,21 +457,23 @@ public class KafkaRaftClientTest { assertEquals(1, retries.size()); } - @Test - public void testResignWillCompleteFetchPurgatory() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testResignWillCompleteFetchPurgatory(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .build(); + .withKip853Rpc(withKip853Rpc) + .build(); context.becomeLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); // send fetch request when become leader int epoch = context.currentEpoch(); - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, context.log.endOffset().offset, epoch, 1000)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, context.log.endOffset().offset, epoch, 1000)); context.client.poll(); // append some record, but the fetch in purgatory will still fail @@ -472,14 +516,19 @@ public class KafkaRaftClientTest { context.assertElectedLeader(currentEpoch, localId); } - @Test - public void testHandleBeginQuorumEpochAfterUserInitiatedResign() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleBeginQuorumEpochAfterUserInitiatedResign( + boolean withKip853Rpc + ) throws Exception { int localId = 0; int remoteId1 = 1; int remoteId2 = 2; Set voters = Utils.mkSet(localId, remoteId1, remoteId2); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); context.becomeLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -497,15 +546,18 @@ public class KafkaRaftClientTest { context.listener.currentLeaderAndEpoch()); } - @Test - public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVoters() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVoters(boolean withKip853Rpc) throws Exception { int localId = 0; - int remoteId1 = 1; - int remoteId2 = 2; - int observerId3 = 3; - Set voters = Utils.mkSet(localId, remoteId1, remoteId2); + ReplicaKey remoteKey1 = replicaKey(1, withKip853Rpc); + ReplicaKey remoteKey2 = replicaKey(2, withKip853Rpc); + ReplicaKey observerKey3 = replicaKey(3, withKip853Rpc); + Set voters = Utils.mkSet(localId, remoteKey1.id(), remoteKey2.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; context.becomeLeader(); @@ -519,7 +571,7 @@ public class KafkaRaftClientTest { assertFalse(context.client.quorum().isResigned()); // Received fetch request from a voter, the fetch timer should be reset. - context.deliverRequest(context.fetchRequest(epoch, remoteId1, 0, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, remoteKey1, 0, 0, 0)); context.pollUntilRequest(); // Since the fetch timer is reset, the leader should not get resigned @@ -529,7 +581,7 @@ public class KafkaRaftClientTest { assertFalse(context.client.quorum().isResigned()); // Received fetch request from another voter, the fetch timer should be reset. - context.deliverRequest(context.fetchRequest(epoch, remoteId2, 0, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, remoteKey2, 0, 0, 0)); context.pollUntilRequest(); // Since the fetch timer is reset, the leader should not get resigned @@ -539,7 +591,7 @@ public class KafkaRaftClientTest { assertFalse(context.client.quorum().isResigned()); // Received fetch request from an observer, but the fetch timer should not be reset. - context.deliverRequest(context.fetchRequest(epoch, observerId3, 0, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, observerKey3, 0, 0, 0)); context.pollUntilRequest(); // After this sleep, the fetch timeout should expire since we don't receive fetch request from the majority voters within fetchTimeoutMs @@ -567,13 +619,16 @@ public class KafkaRaftClientTest { assertFalse(context.client.quorum().isResigned()); } - @Test - public void testElectionTimeoutAfterUserInitiatedResign() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; Set voters = Utils.mkSet(localId, otherNodeId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); context.becomeLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -586,12 +641,9 @@ public class KafkaRaftClientTest { context.pollUntilRequest(); RaftRequest.Outbound request = context.assertSentEndQuorumEpochRequest(resignedEpoch, otherNodeId); - EndQuorumEpochResponseData response = EndQuorumEpochResponse.singletonResponse( - Errors.NONE, - context.metadataPartition, - Errors.NONE, + EndQuorumEpochResponseData response = context.endEpochResponse( resignedEpoch, - localId + OptionalInt.of(localId) ); context.deliverResponse(request.correlationId(), request.destination(), response); @@ -603,10 +655,14 @@ public class KafkaRaftClientTest { assertFalse(context.channel.hasSentRequests()); // Any `Fetch` received in the resigned state should result in a NOT_LEADER error. - context.deliverRequest(context.fetchRequest(1, -1, 0, 0, 0)); + ReplicaKey observer = replicaKey(-1, withKip853Rpc); + context.deliverRequest(context.fetchRequest(1, observer, 0, 0, 0)); context.pollUntilResponse(); - context.assertSentFetchPartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER, - resignedEpoch, OptionalInt.of(localId)); + context.assertSentFetchPartitionResponse( + Errors.NOT_LEADER_OR_FOLLOWER, + resignedEpoch, + OptionalInt.of(localId) + ); // After the election timer, we should become a candidate. context.time.sleep(2L * context.electionTimeoutMs()); @@ -676,7 +732,7 @@ public class KafkaRaftClientTest { Set voters = Utils.mkSet(localId, 1, 2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withVotedCandidate(2, ReplicaKey.of(localId, Optional.empty())) + .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) .build(); context.assertVotedCandidate(2, localId); assertEquals(0L, context.log.endOffset().offset); @@ -774,27 +830,34 @@ public class KafkaRaftClientTest { Arrays.asList(firstNodeId, localId), record.key(), record.value()); } - @Test - public void testHandleBeginQuorumRequest() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleBeginQuorumRequest(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int votedCandidateEpoch = 2; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withVotedCandidate(votedCandidateEpoch, ReplicaKey.of(otherNodeId, Optional.empty())) + .withVotedCandidate(votedCandidateEpoch, otherNodeKey) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.beginEpochRequest(votedCandidateEpoch, otherNodeId)); + context.deliverRequest(context.beginEpochRequest(votedCandidateEpoch, otherNodeKey.id())); context.pollUntilResponse(); - context.assertElectedLeader(votedCandidateEpoch, otherNodeId); + context.assertElectedLeader(votedCandidateEpoch, otherNodeKey.id()); - context.assertSentBeginQuorumEpochResponse(Errors.NONE, votedCandidateEpoch, OptionalInt.of(otherNodeId)); + context.assertSentBeginQuorumEpochResponse( + Errors.NONE, + votedCandidateEpoch, + OptionalInt.of(otherNodeKey.id()) + ); } - @Test - public void testHandleBeginQuorumResponse() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; int leaderEpoch = 2; @@ -802,6 +865,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(leaderEpoch, localId) + .withKip853Rpc(withKip853Rpc) .build(); context.deliverRequest(context.beginEpochRequest(leaderEpoch + 1, otherNodeId)); @@ -810,8 +874,9 @@ public class KafkaRaftClientTest { context.assertElectedLeader(leaderEpoch + 1, otherNodeId); } - @Test - public void testEndQuorumIgnoredAsCandidateIfOlderEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; int epoch = 5; @@ -821,6 +886,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(jitterMs)) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); // Sleep a little to ensure that we become a candidate @@ -828,8 +894,13 @@ public class KafkaRaftClientTest { context.client.poll(); context.assertVotedCandidate(epoch, localId); - context.deliverRequest(context.endEpochRequest(epoch - 2, otherNodeId, - Collections.singletonList(localId))); + context.deliverRequest( + context.endEpochRequest( + epoch - 2, + otherNodeId, + Collections.singletonList(context.localReplicaKey()) + ) + ); context.client.poll(); context.assertSentEndQuorumEpochResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.empty()); @@ -850,18 +921,25 @@ public class KafkaRaftClientTest { context.assertVotedCandidate(epoch + 1, localId); } - @Test - public void testEndQuorumIgnoredAsLeaderIfOlderEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; int voter2 = localId + 1; - int voter3 = localId + 2; - int epoch = 7; - Set voters = Utils.mkSet(localId, voter2, voter3); + ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); + Set voters = Utils.mkSet(localId, voter2, voter3.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(6) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // One of the voters may have sent EndQuorumEpoch from an earlier epoch - context.deliverRequest(context.endEpochRequest(epoch - 2, voter2, Arrays.asList(localId, voter3))); + context.deliverRequest( + context.endEpochRequest(epoch - 2, voter2, Arrays.asList(context.localReplicaKey(), voter3)) + ); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.of(localId)); @@ -872,20 +950,29 @@ public class KafkaRaftClientTest { context.assertElectedLeader(epoch, localId); } - @Test - public void testEndQuorumStartsNewElectionImmediatelyIfFollowerUnattached() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testEndQuorumStartsNewElectionImmediatelyIfFollowerUnattached( + boolean withKip853Rpc + ) throws Exception { int localId = 0; int voter2 = localId + 1; - int voter3 = localId + 2; + ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); int epoch = 2; - Set voters = Utils.mkSet(localId, voter2, voter3); + Set voters = Utils.mkSet(localId, voter2, voter3.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.endEpochRequest(epoch, voter2, - Arrays.asList(localId, voter3))); + context.deliverRequest( + context.endEpochRequest( + epoch, + voter2, + Arrays.asList(context.localReplicaKey(), voter3) + ) + ); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.NONE, epoch, OptionalInt.of(voter2)); @@ -895,8 +982,9 @@ public class KafkaRaftClientTest { context.assertVotedCandidate(epoch + 1, localId); } - @Test - public void testAccumulatorClearedAfterBecomingFollower() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; int lingerMs = 50; @@ -910,6 +998,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -925,12 +1014,13 @@ public class KafkaRaftClientTest { Mockito.verify(memoryPool, Mockito.times(2)).release(buffer); } - @Test - public void testAccumulatorClearedAfterBecomingVoted() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int lingerMs = 50; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); @@ -940,6 +1030,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -947,20 +1038,22 @@ public class KafkaRaftClientTest { int epoch = context.currentEpoch(); assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch, - context.log.endOffset().offset)); + context.deliverRequest( + context.voteRequest(epoch + 1, otherNodeKey, epoch, context.log.endOffset().offset) + ); context.pollUntilResponse(); - context.assertVotedCandidate(epoch + 1, otherNodeId); + context.assertVotedCandidate(epoch + 1, otherNodeKey.id()); Mockito.verify(memoryPool, Mockito.times(2)).release(buffer); } - @Test - public void testAccumulatorClearedAfterBecomingUnattached() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int lingerMs = 50; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); @@ -970,6 +1063,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) .withMemoryPool(memoryPool) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -977,7 +1071,7 @@ public class KafkaRaftClientTest { int epoch = context.currentEpoch(); assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch, 0L)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 0L)); context.pollUntilResponse(); context.assertUnknownLeader(epoch + 1); @@ -1053,8 +1147,9 @@ public class KafkaRaftClientTest { assertEquals(3L, context.log.endOffset().offset); } - @Test - public void testHandleEndQuorumRequest() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { int localId = 0; int oldLeaderId = 1; int leaderEpoch = 2; @@ -1062,10 +1157,16 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(leaderEpoch, oldLeaderId) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.endEpochRequest(leaderEpoch, oldLeaderId, - Collections.singletonList(localId))); + context.deliverRequest( + context.endEpochRequest( + leaderEpoch, + oldLeaderId, + Collections.singletonList(context.localReplicaKey()) + ) + ); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.NONE, leaderEpoch, OptionalInt.of(oldLeaderId)); @@ -1074,23 +1175,30 @@ public class KafkaRaftClientTest { context.assertVotedCandidate(leaderEpoch + 1, localId); } - @Test - public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean withKip853Rpc) throws Exception { int localId = 0; - int oldLeaderId = 1; + ReplicaKey oldLeaderKey = replicaKey(1, withKip853Rpc); int leaderEpoch = 2; - int preferredNextLeader = 3; - Set voters = Utils.mkSet(localId, oldLeaderId, preferredNextLeader); + ReplicaKey preferredNextLeader = replicaKey(3, withKip853Rpc); + Set voters = Utils.mkSet(localId, oldLeaderKey.id(), preferredNextLeader.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withElectedLeader(leaderEpoch, oldLeaderId) + .withElectedLeader(leaderEpoch, oldLeaderKey.id()) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.endEpochRequest(leaderEpoch, oldLeaderId, - Arrays.asList(preferredNextLeader, localId))); + context.deliverRequest( + context.endEpochRequest( + leaderEpoch, + oldLeaderKey.id(), + Arrays.asList(preferredNextLeader, context.localReplicaKey()) + ) + ); context.pollUntilResponse(); - context.assertSentEndQuorumEpochResponse(Errors.NONE, leaderEpoch, OptionalInt.of(oldLeaderId)); + context.assertSentEndQuorumEpochResponse(Errors.NONE, leaderEpoch, OptionalInt.of(oldLeaderKey.id())); // The election won't trigger by one round retry backoff context.time.sleep(1); @@ -1149,38 +1257,42 @@ public class KafkaRaftClientTest { context.assertElectedLeader(epoch, localId); } - @Test - public void testHandleValidVoteRequestAsFollower() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleValidVoteRequestAsFollower(boolean withKip853Rpc) throws Exception { int localId = 0; int epoch = 2; - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.voteRequest(epoch, otherNodeId, epoch - 1, 1)); + context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - context.assertVotedCandidate(epoch, otherNodeId); + context.assertVotedCandidate(epoch, otherNodeKey.id()); } - @Test - public void testHandleVoteRequestAsFollowerWithElectedLeader() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleVoteRequestAsFollowerWithElectedLeader(boolean withKip853Rpc) throws Exception { int localId = 0; int epoch = 2; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int electedLeaderId = 3; - Set voters = Utils.mkSet(localId, otherNodeId, electedLeaderId); + Set voters = Utils.mkSet(localId, otherNodeKey.id(), electedLeaderId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, electedLeaderId) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.voteRequest(epoch, otherNodeId, epoch - 1, 1)); + context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeaderId), false); @@ -1188,72 +1300,84 @@ public class KafkaRaftClientTest { context.assertElectedLeader(epoch, electedLeaderId); } - @Test - public void testHandleVoteRequestAsFollowerWithVotedCandidate() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853Rpc) throws Exception { int localId = 0; int epoch = 2; - int otherNodeId = 1; - int votedCandidateId = 3; - Set voters = Utils.mkSet(localId, otherNodeId, votedCandidateId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + ReplicaKey votedCandidateKey = replicaKey(3, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id(), votedCandidateKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withVotedCandidate(epoch, ReplicaKey.of(votedCandidateId, Optional.empty())) + .withVotedCandidate(epoch, votedCandidateKey) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.voteRequest(epoch, otherNodeId, epoch - 1, 1)); + context.deliverRequest(context.voteRequest(epoch, otherNodeKey, epoch - 1, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), false); - context.assertVotedCandidate(epoch, votedCandidateId); + context.assertVotedCandidate(epoch, votedCandidateKey.id()); } - @Test - public void testHandleInvalidVoteRequestWithOlderEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; int epoch = 2; - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.voteRequest(epoch - 1, otherNodeId, epoch - 2, 1)); + context.deliverRequest(context.voteRequest(epoch - 1, otherNodeKey, epoch - 2, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.empty(), false); context.assertUnknownLeader(epoch); } - @Test - public void testHandleInvalidVoteRequestAsObserver() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleVoteRequestAsObserver(boolean withKip853Rpc) throws Exception { int localId = 0; int epoch = 2; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int otherNodeId2 = 2; - Set voters = Utils.mkSet(otherNodeId, otherNodeId2); + Set voters = Utils.mkSet(otherNodeKey.id(), otherNodeId2); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) + .withKip853Rpc(withKip853Rpc) .build(); - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch, 1)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); - context.assertVotedCandidate(epoch + 1, otherNodeId); + context.assertVotedCandidate(epoch + 1, otherNodeKey.id()); } - @Test - public void testLeaderIgnoreVoteRequestOnSameEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int leaderEpoch = 2; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, leaderEpoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(2) + .withKip853Rpc(withKip853Rpc) + .build(); - context.deliverRequest(context.voteRequest(leaderEpoch, otherNodeId, leaderEpoch - 1, 1)); + context.becomeLeader(); + int leaderEpoch = context.currentEpoch(); + + context.deliverRequest(context.voteRequest(leaderEpoch, otherNodeKey, leaderEpoch - 1, 1)); context.client.poll(); @@ -1261,14 +1385,20 @@ public class KafkaRaftClientTest { context.assertElectedLeader(leaderEpoch, localId); } - @Test - public void testListenerCommitCallbackAfterLeaderWrite() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // First poll has no high watermark advance context.client.poll(); @@ -1277,7 +1407,7 @@ public class KafkaRaftClientTest { // Let follower send a fetch to initialize the high watermark, // note the offset 0 would be a control message for becoming the leader - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 1L, epoch, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 1L, epoch, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); assertEquals(OptionalLong.of(1L), context.client.highWatermark()); @@ -1288,26 +1418,28 @@ public class KafkaRaftClientTest { assertEquals(OptionalLong.of(0L), context.listener.lastCommitOffset()); // Let the follower send a fetch, it should advance the high watermark - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 1L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 1L, epoch, 500)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); assertEquals(OptionalLong.of(1L), context.client.highWatermark()); assertEquals(OptionalLong.of(0L), context.listener.lastCommitOffset()); // Let the follower send another fetch from offset 4 - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 4L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 4L, epoch, 500)); context.pollUntil(() -> context.client.highWatermark().equals(OptionalLong.of(4L))); assertEquals(records, context.listener.commitWithLastOffset(offset)); } - @Test - public void testLeaderImmediatelySendsDivergingEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(5) + .withKip853Rpc(withKip853Rpc) .appendToLog(1, Arrays.asList("a", "b", "c")) .appendToLog(3, Arrays.asList("d", "e", "f")) .appendToLog(5, Arrays.asList("g", "h", "i")) @@ -1318,7 +1450,7 @@ public class KafkaRaftClientTest { int epoch = context.currentEpoch(); // Send a fetch request for an end offset and epoch which has diverged - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 6, 2, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 6, 2, 500)); context.client.poll(); // Expect that the leader replies immediately with a diverging epoch @@ -1330,20 +1462,22 @@ public class KafkaRaftClientTest { assertEquals(3, partitionResponse.divergingEpoch().endOffset()); } - @Test - public void testCandidateIgnoreVoteRequestOnSameEpoch() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCandidateIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int leaderEpoch = 2; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, Optional.empty())) + .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + .withKip853Rpc(withKip853Rpc) .build(); context.pollUntilRequest(); - context.deliverRequest(context.voteRequest(leaderEpoch, otherNodeId, leaderEpoch - 1, 1)); + context.deliverRequest(context.voteRequest(leaderEpoch, otherNodeKey, leaderEpoch - 1, 1)); context.client.poll(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), false); context.assertVotedCandidate(leaderEpoch, localId); @@ -1695,37 +1829,38 @@ public class KafkaRaftClientTest { assertFalse(context.channel.hasSentRequests()); } - @Test - public void testInvalidFetchRequest() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInvalidFetchRequest(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); - context.deliverRequest(context.fetchRequest( - epoch, otherNodeId, -5L, 0, 0)); + context.becomeLeader(); + int epoch = context.currentEpoch(); + + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, -5L, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); - context.deliverRequest(context.fetchRequest( - epoch, otherNodeId, 0L, -1, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0L, -1, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); - context.deliverRequest(context.fetchRequest( - epoch, otherNodeId, 0L, epoch + 1, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0L, epoch + 1, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); - context.deliverRequest(context.fetchRequest( - epoch + 1, otherNodeId, 0L, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch + 1, otherNodeKey, 0L, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.UNKNOWN_LEADER_EPOCH, epoch, OptionalInt.of(localId)); - context.deliverRequest(context.fetchRequest( - epoch, otherNodeId, 0L, 0, -1)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0L, 0, -1)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); } @@ -1735,9 +1870,9 @@ public class KafkaRaftClientTest { @ApiKeyVersionsSource(apiKey = ApiKeys.FETCH) public void testLeaderStateUpdateWithDifferentFetchRequestVersions(short version) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, false); int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); @@ -1747,7 +1882,7 @@ public class KafkaRaftClientTest { assertEquals(1L, context.log.endOffset().offset); // Now we will advance the high watermark with a follower fetch request. - FetchRequestData fetchRequestData = context.fetchRequest(epoch, otherNodeId, 1L, epoch, 0); + FetchRequestData fetchRequestData = context.fetchRequest(epoch, otherNodeKey, 1L, epoch, 0); FetchRequestData request = new FetchRequest.SimpleBuilder(fetchRequestData).build(version).data(); assertEquals((version < 15) ? 1 : -1, fetchRequestData.replicaId()); assertEquals((version < 15) ? -1 : 1, fetchRequestData.replicaState().replicaId()); @@ -1757,78 +1892,187 @@ public class KafkaRaftClientTest { assertEquals(OptionalLong.of(1L), context.client.highWatermark()); } - @Test - public void testFetchRequestClusterIdValidation() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // valid cluster id is accepted - context.deliverRequest(context.fetchRequest( - epoch, context.clusterId.toString(), otherNodeId, -5L, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, -5L, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); // null cluster id is accepted - context.deliverRequest(context.fetchRequest( - epoch, null, otherNodeId, -5L, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, null, otherNodeKey, -5L, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); // empty cluster id is rejected - context.deliverRequest(context.fetchRequest( - epoch, "", otherNodeId, -5L, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, "", otherNodeKey, -5L, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INCONSISTENT_CLUSTER_ID); // invalid cluster id is rejected - context.deliverRequest(context.fetchRequest( - epoch, "invalid-uuid", otherNodeId, -5L, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, "invalid-uuid", otherNodeKey, -5L, 0, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.INCONSISTENT_CLUSTER_ID); } - @Test - public void testVoteRequestClusterIdValidation() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testVoteRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // valid cluster id is accepted - context.deliverRequest(context.voteRequest(epoch, localId, 0, 0)); + context.deliverRequest(context.voteRequest(epoch, otherNodeKey, 0, 0)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), false); // null cluster id is accepted - context.deliverRequest(context.voteRequest(epoch, localId, 0, 0)); + context.deliverRequest(context.voteRequest(null, epoch, otherNodeKey, 0, 0)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), false); // empty cluster id is rejected - context.deliverRequest(context.voteRequest("", epoch, localId, 0, 0)); + context.deliverRequest(context.voteRequest("", epoch, otherNodeKey, 0, 0)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.INCONSISTENT_CLUSTER_ID); // invalid cluster id is rejected - context.deliverRequest(context.voteRequest("invalid-uuid", epoch, localId, 0, 0)); + context.deliverRequest(context.voteRequest("invalid-uuid", epoch, otherNodeKey, 0, 0)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.INCONSISTENT_CLUSTER_ID); } @Test - public void testBeginQuorumEpochRequestClusterIdValidation() throws Exception { + public void testInvalidVoterReplicaVoteRequest() throws Exception { + int localId = 0; + ReplicaKey otherNodeKey = replicaKey(1, true); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(true) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); + + // invalid voter id is rejected + context.deliverRequest( + context.voteRequest( + context.clusterId.toString(), + epoch + 1, + otherNodeKey, + ReplicaKey.of(10, Uuid.randomUuid()), + epoch, + 100 + ) + ); + context.pollUntilResponse(); + context.assertSentVoteResponse(Errors.INVALID_VOTER_KEY, epoch + 1, OptionalInt.empty(), false); + + // invalid voter directory id is rejected + context.deliverRequest( + context.voteRequest( + context.clusterId.toString(), + epoch + 2, + otherNodeKey, + ReplicaKey.of(0, Uuid.randomUuid()), + epoch, + 100 + ) + ); + context.pollUntilResponse(); + context.assertSentVoteResponse(Errors.INVALID_VOTER_KEY, epoch + 2, OptionalInt.empty(), false); + } + + @Test + public void testInvalidVoterReplicaBeginQuorumEpochRequest() throws Exception { + int localId = 0; + int voter1 = localId; + int voter2 = localId + 1; + int voter3 = localId + 2; + int epoch = 5; + Set voters = Utils.mkSet(voter1, voter2, voter3); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(epoch - 1) + .withKip853Rpc(true) + .build(); + context.assertUnknownLeader(epoch - 1); + + // Leader voter3 sends a begin quorum epoch request with incorrect voter id + context.deliverRequest( + context.beginEpochRequest( + context.clusterId.toString(), + epoch, + voter3, + ReplicaKey.of(10, Uuid.randomUuid()) + ) + ); + context.pollUntilResponse(); + context.assertSentBeginQuorumEpochResponse(Errors.INVALID_VOTER_KEY, epoch, OptionalInt.of(voter3)); + context.assertElectedLeader(epoch, voter3); + + // Leader voter3 sends a begin quorum epoch request with incorrect voter directory id + context.deliverRequest( + context.beginEpochRequest( + context.clusterId.toString(), + epoch, + voter3, + ReplicaKey.of(localId, Uuid.randomUuid()) + ) + ); + context.pollUntilResponse(); + context.assertSentBeginQuorumEpochResponse(Errors.INVALID_VOTER_KEY, epoch, OptionalInt.of(voter3)); + context.assertElectedLeader(epoch, voter3); + + // Leader voter3 sends a begin quorum epoch request with incorrect voter directory id + context.deliverRequest( + context.beginEpochRequest( + context.clusterId.toString(), + epoch, + voter3, + context.localReplicaKey() + ) + ); + context.pollUntilResponse(); + context.assertSentBeginQuorumEpochResponse(Errors.NONE, epoch, OptionalInt.of(voter3)); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; - int epoch = 5; Set voters = Utils.mkSet(localId, otherNodeId); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // valid cluster id is accepted context.deliverRequest(context.beginEpochRequest(context.clusterId.toString(), epoch, localId)); @@ -1851,95 +2095,130 @@ public class KafkaRaftClientTest { context.assertSentBeginQuorumEpochResponse(Errors.INCONSISTENT_CLUSTER_ID); } - @Test - public void testEndQuorumEpochRequestClusterIdValidation() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // valid cluster id is accepted - context.deliverRequest(context.endEpochRequest(context.clusterId.toString(), epoch, localId, Collections.singletonList(otherNodeId))); + context.deliverRequest(context.endEpochRequest(epoch, localId, Collections.singletonList(otherNodeKey))); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.NONE, epoch, OptionalInt.of(localId)); // null cluster id is accepted - context.deliverRequest(context.endEpochRequest(epoch, localId, Collections.singletonList(otherNodeId))); + context.deliverRequest(context.endEpochRequest(null, epoch, localId, Collections.singletonList(otherNodeKey))); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.NONE, epoch, OptionalInt.of(localId)); // empty cluster id is rejected - context.deliverRequest(context.endEpochRequest("", epoch, localId, Collections.singletonList(otherNodeId))); + context.deliverRequest(context.endEpochRequest("", epoch, localId, Collections.singletonList(otherNodeKey))); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.INCONSISTENT_CLUSTER_ID); // invalid cluster id is rejected - context.deliverRequest(context.endEpochRequest("invalid-uuid", epoch, localId, Collections.singletonList(otherNodeId))); + context.deliverRequest(context.endEpochRequest("invalid-uuid", epoch, localId, Collections.singletonList(otherNodeKey))); context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.INCONSISTENT_CLUSTER_ID); } - @Test - public void testLeaderAcceptVoteFromNonVoter() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLeaderAcceptVoteFromNonVoter(boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; - int epoch = 5; Set voters = Utils.mkSet(localId, otherNodeId); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); - int nonVoterId = 2; - context.deliverRequest(context.voteRequest(epoch - 1, nonVoterId, 0, 0)); + context.becomeLeader(); + int epoch = context.currentEpoch(); + + ReplicaKey nonVoterKey = replicaKey(2, withKip853Rpc); + context.deliverRequest(context.voteRequest(epoch - 1, nonVoterKey, 0, 0)); context.client.poll(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.of(localId), false); - context.deliverRequest(context.voteRequest(epoch, nonVoterId, 0, 0)); + context.deliverRequest(context.voteRequest(epoch, nonVoterKey, 0, 0)); context.client.poll(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), false); } - @Test - public void testInvalidVoteRequest() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInvalidVoteRequest(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withElectedLeader(epoch, otherNodeId) + .withElectedLeader(epoch, otherNodeKey.id()) + .withKip853Rpc(withKip853Rpc) .build(); - context.assertElectedLeader(epoch, otherNodeId); + context.assertElectedLeader(epoch, otherNodeKey.id()); - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, 0, -5L)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, 0, -5L)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(otherNodeId), false); - context.assertElectedLeader(epoch, otherNodeId); + context.assertSentVoteResponse( + Errors.INVALID_REQUEST, + epoch, + OptionalInt.of(otherNodeKey.id()), + false + ); + context.assertElectedLeader(epoch, otherNodeKey.id()); - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, -1, 0L)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, -1, 0L)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(otherNodeId), false); - context.assertElectedLeader(epoch, otherNodeId); + context.assertSentVoteResponse( + Errors.INVALID_REQUEST, + epoch, + OptionalInt.of(otherNodeKey.id()), + false + ); + context.assertElectedLeader(epoch, otherNodeKey.id()); - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch + 1, 0L)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch + 1, 0L)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(otherNodeId), false); - context.assertElectedLeader(epoch, otherNodeId); + context.assertSentVoteResponse( + Errors.INVALID_REQUEST, + epoch, + OptionalInt.of(otherNodeKey.id()), + false + ); + context.assertElectedLeader(epoch, otherNodeKey.id()); } - @Test - public void testPurgatoryFetchTimeout() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately int maxWaitTimeMs = 500; - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 1L, epoch, maxWaitTimeMs)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 1L, epoch, maxWaitTimeMs)); context.client.poll(); assertEquals(0, context.channel.drainSendQueue().size()); @@ -1950,17 +2229,23 @@ public class KafkaRaftClientTest { assertEquals(0, fetchedRecords.sizeInBytes()); } - @Test - public void testPurgatoryFetchSatisfiedByWrite() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 1L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 1L, epoch, 500)); context.client.poll(); assertEquals(0, context.channel.drainSendQueue().size()); @@ -1973,19 +2258,25 @@ public class KafkaRaftClientTest { RaftClientTestContext.assertMatchingRecords(appendRecords, fetchedRecords); } - @Test - public void testPurgatoryFetchCompletedByFollowerTransition() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rpc) throws Exception { int localId = 0; int voter1 = localId; - int voter2 = localId + 1; + ReplicaKey voterKey2 = replicaKey(localId + 1, withKip853Rpc); int voter3 = localId + 2; - int epoch = 5; - Set voters = Utils.mkSet(voter1, voter2, voter3); + Set voters = Utils.mkSet(voter1, voterKey2.id(), voter3); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(4) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately - context.deliverRequest(context.fetchRequest(epoch, voter2, 1L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, voterKey2, 1L, epoch, 500)); context.client.poll(); assertTrue(context.channel.drainSendQueue().stream() .noneMatch(msg -> msg.data() instanceof FetchResponseData)); @@ -2039,8 +2330,11 @@ public class KafkaRaftClientTest { context.assertVotedCandidate(epoch + 1, localId); } - @Test - public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( + boolean withKip853Rpc + ) throws Exception { int localId = 0; int voter1 = localId; int voter2 = localId + 1; @@ -2051,6 +2345,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, voter2) + .withKip853Rpc(withKip853Rpc) .build(); context.assertElectedLeader(epoch, voter2); @@ -2077,8 +2372,9 @@ public class KafkaRaftClientTest { context.assertElectedLeader(epoch + 1, voter3); } - @Test - public void testVoteResponseIgnoredAfterBecomingFollower() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) throws Exception { int localId = 0; int voter1 = localId; int voter2 = localId + 1; @@ -2088,6 +2384,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); context.assertUnknownLeader(epoch - 1); @@ -2216,14 +2513,19 @@ public class KafkaRaftClientTest { context.assertElectedLeader(epoch, leaderId); } - @Test - public void testLeaderGracefulShutdown() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; - int epoch = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // Now shutdown int shutdownTimeoutMs = 5000; @@ -2238,16 +2540,16 @@ public class KafkaRaftClientTest { context.pollUntilRequest(); assertTrue(context.client.isShuttingDown()); assertTrue(context.client.isRunning()); - context.assertSentEndQuorumEpochRequest(1, otherNodeId); + context.assertSentEndQuorumEpochRequest(1, otherNodeKey.id()); // We should still be able to handle vote requests during graceful shutdown // in order to help the new leader get elected - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch, 1L)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1L)); context.client.poll(); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); // Graceful shutdown completes when a new leader is elected - context.deliverRequest(context.beginEpochRequest(2, otherNodeId)); + context.deliverRequest(context.beginEpochRequest(2, otherNodeKey.id())); TestUtils.waitForCondition(() -> { context.client.poll(); @@ -2258,15 +2560,20 @@ public class KafkaRaftClientTest { assertNull(shutdownFuture.get()); } - @Test - public void testEndQuorumEpochSentBasedOnFetchOffset() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) throws Exception { int localId = 0; - int closeFollower = 2; - int laggingFollower = 1; - int epoch = 1; - Set voters = Utils.mkSet(localId, closeFollower, laggingFollower); + ReplicaKey closeFollower = replicaKey(2, withKip853Rpc); + ReplicaKey laggingFollower = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, closeFollower.id(), laggingFollower.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); // The lagging follower fetches first context.deliverRequest(context.fetchRequest(1, laggingFollower, 1L, epoch, 0)); @@ -2293,24 +2600,25 @@ public class KafkaRaftClientTest { context.collectEndQuorumRequests( epoch, - Utils.mkSet(closeFollower, laggingFollower), - Optional.of(Arrays.asList(closeFollower, laggingFollower)) + Utils.mkSet(closeFollower.id(), laggingFollower.id()), + Optional.of(Arrays.asList(closeFollower.id(), laggingFollower.id())) ); } - @Test - public void testDescribeQuorumNonLeader() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testDescribeQuorumNonLeader(boolean withKip853Rpc) throws Exception { int localId = 0; - int voter2 = localId + 1; - int voter3 = localId + 2; + ReplicaKey voter2 = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc); int epoch = 2; - Set voters = Utils.mkSet(localId, voter2, voter3); + Set voters = Utils.mkSet(localId, voter2.id(), voter3.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) .build(); - context.deliverRequest(DescribeQuorumRequest.singletonRequest(context.metadataPartition)); + context.deliverRequest(context.describeQuorumRequest()); context.pollUntilResponse(); DescribeQuorumResponseData responseData = context.collectDescribeQuorumResponse(); @@ -2328,15 +2636,20 @@ public class KafkaRaftClientTest { assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.message(), partitionData.errorMessage()); } - @Test - public void testDescribeQuorum() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testDescribeQuorum(boolean withKip853Rpc) throws Exception { int localId = 0; - int closeFollower = 2; - int laggingFollower = 1; - int epoch = 1; - Set voters = Utils.mkSet(localId, closeFollower, laggingFollower); + ReplicaKey closeFollower = replicaKey(2, withKip853Rpc); + ReplicaKey laggingFollower = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, closeFollower.id(), laggingFollower.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); long laggingFollowerFetchTime = context.time.milliseconds(); context.deliverRequest(context.fetchRequest(1, laggingFollower, 1L, epoch, 0)); @@ -2353,7 +2666,7 @@ public class KafkaRaftClientTest { context.assertSentFetchPartitionResponse(3L, epoch); // Create observer - int observerId = 3; + ReplicaKey observerId = replicaKey(3, withKip853Rpc); context.time.sleep(100); long observerFetchTime = context.time.milliseconds(); context.deliverRequest(context.fetchRequest(epoch, observerId, 0L, 0, 0)); @@ -2361,35 +2674,35 @@ public class KafkaRaftClientTest { context.assertSentFetchPartitionResponse(3L, epoch); context.time.sleep(100); - context.deliverRequest(DescribeQuorumRequest.singletonRequest(context.metadataPartition)); + context.deliverRequest(context.describeQuorumRequest()); context.pollUntilResponse(); context.assertSentDescribeQuorumResponse(localId, epoch, 3L, Arrays.asList( new ReplicaState() .setReplicaId(localId) - .setReplicaDirectoryId(Uuid.ZERO_UUID) + .setReplicaDirectoryId(ReplicaKey.NO_DIRECTORY_ID) // As we are appending the records directly to the log, // the leader end offset hasn't been updated yet. .setLogEndOffset(3L) .setLastFetchTimestamp(context.time.milliseconds()) .setLastCaughtUpTimestamp(context.time.milliseconds()), new ReplicaState() - .setReplicaId(laggingFollower) - .setReplicaDirectoryId(Uuid.ZERO_UUID) + .setReplicaId(laggingFollower.id()) + .setReplicaDirectoryId(ReplicaKey.NO_DIRECTORY_ID) .setLogEndOffset(1L) .setLastFetchTimestamp(laggingFollowerFetchTime) .setLastCaughtUpTimestamp(laggingFollowerFetchTime), new ReplicaState() - .setReplicaId(closeFollower) - .setReplicaDirectoryId(Uuid.ZERO_UUID) + .setReplicaId(closeFollower.id()) + .setReplicaDirectoryId(ReplicaKey.NO_DIRECTORY_ID) .setLogEndOffset(3L) .setLastFetchTimestamp(closeFollowerFetchTime) .setLastCaughtUpTimestamp(closeFollowerFetchTime)), singletonList( new ReplicaState() - .setReplicaId(observerId) - .setReplicaDirectoryId(Uuid.ZERO_UUID) + .setReplicaId(observerId.id()) + .setReplicaDirectoryId(observerId.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(0L) .setLastFetchTimestamp(observerFetchTime) .setLastCaughtUpTimestamp(-1L))); @@ -2585,16 +2898,18 @@ public class KafkaRaftClientTest { assertEquals(OptionalLong.of(2L), context.client.highWatermark()); } - @Test - public void testFetchShouldBeTreatedAsLeaderAcknowledgement() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); context.time.sleep(context.electionTimeoutMs()); @@ -2605,8 +2920,7 @@ public class KafkaRaftClientTest { // We send BeginEpoch, but it gets lost and the destination finds the leader through the Fetch API context.assertSentBeginQuorumEpochRequest(epoch, 1); - context.deliverRequest(context.fetchRequest( - epoch, otherNodeId, 0L, 0, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0L, 0, 500)); context.client.poll(); @@ -2620,12 +2934,15 @@ public class KafkaRaftClientTest { assertEquals(0, sentMessages.size()); } - @Test - public void testLeaderAppendSingleMemberQuorum() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exception { int localId = 0; Set voters = Collections.singleton(localId); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) + .build(); long now = context.time.milliseconds(); context.pollUntil(() -> context.log.endOffset().offset == 1L); @@ -2647,7 +2964,7 @@ public class KafkaRaftClientTest { assertEquals(OptionalLong.of(4L), context.client.highWatermark()); // Now try reading it - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); List batches = new ArrayList<>(2); boolean appended = true; @@ -2661,7 +2978,7 @@ public class KafkaRaftClientTest { lastFetchedEpoch = lastBatch.partitionLeaderEpoch(); } - context.deliverRequest(context.fetchRequest(1, otherNodeId, fetchOffset, lastFetchedEpoch, 0)); + context.deliverRequest(context.fetchRequest(1, otherNodeKey, fetchOffset, lastFetchedEpoch, 0)); context.pollUntilResponse(); MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, 1, OptionalInt.of(localId)); @@ -2867,13 +3184,17 @@ public class KafkaRaftClientTest { assertThrows(ClusterAuthorizationException.class, context.client::poll); } - @Test - public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmptyLog() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmptyLog( + boolean withKip853Rpc + ) throws Exception { int localId = 0; - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -2891,7 +3212,7 @@ public class KafkaRaftClientTest { // Deliver a fetch from the other voter. The high watermark will not // be exposed until it is able to reach the start of the leader epoch, // so we are unable to deliver committed data or fire `handleLeaderChange`. - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0L, 0, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0L, 0, 0)); context.client.poll(); assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); assertEquals(OptionalLong.empty(), context.listener.lastCommitOffset()); @@ -2900,7 +3221,7 @@ public class KafkaRaftClientTest { // watermark advances and we can start sending committed data to the // listener. Note that the `LeaderChange` control record is included // in the committed batches. - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 1L, epoch, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 1L, epoch, 0)); context.client.poll(); assertEquals(OptionalLong.of(0), context.listener.lastCommitOffset()); @@ -2910,12 +3231,15 @@ public class KafkaRaftClientTest { assertEquals(0, context.listener.claimedEpochStartOffset(epoch)); } - @Test - public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( + boolean withKip853Rpc + ) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); List batch1 = Arrays.asList("1", "2", "3"); List batch2 = Arrays.asList("4", "5", "6"); @@ -2927,6 +3251,7 @@ public class KafkaRaftClientTest { .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -2944,7 +3269,7 @@ public class KafkaRaftClientTest { // Deliver a fetch from the other voter. The high watermark will not // be exposed until it is able to reach the start of the leader epoch, // so we are unable to deliver committed data or fire `handleLeaderChange`. - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 3L, 1, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 3L, 1, 500)); context.client.poll(); assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); assertEquals(OptionalLong.empty(), context.listener.lastCommitOffset()); @@ -2953,7 +3278,7 @@ public class KafkaRaftClientTest { // watermark advances and we can start sending committed data to the // listener. Note that the `LeaderChange` control record is included // in the committed batches. - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 10L, epoch, 500)); context.pollUntil(() -> { int index = 0; for (Batch batch : context.listener.committedBatches()) { @@ -2975,12 +3300,13 @@ public class KafkaRaftClientTest { assertEquals(9, context.listener.claimedEpochStartOffset(epoch)); } - @Test - public void testLateRegisteredListenerCatchesUp() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Exception { int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int epoch = 5; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); List batch1 = Arrays.asList("1", "2", "3"); List batch2 = Arrays.asList("4", "5", "6"); @@ -2991,6 +3317,7 @@ public class KafkaRaftClientTest { .appendToLog(1, batch2) .appendToLog(2, batch3) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); context.becomeLeader(); @@ -2998,7 +3325,7 @@ public class KafkaRaftClientTest { assertEquals(10L, context.log.endOffset().offset); // Let the initial listener catch up - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 10L, epoch, 0)); context.pollUntil(() -> OptionalInt.of(epoch).equals(context.listener.currentClaimedEpoch())); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); assertEquals(OptionalLong.of(9L), context.listener.lastCommitOffset()); @@ -3114,34 +3441,36 @@ public class KafkaRaftClientTest { assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); } - @Test - public void testHandleCommitCallbackFiresInVotedState() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) throws Exception { // This test verifies that the state machine can still catch up even while // an election is in progress as long as the high watermark is known. int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int epoch = 7; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(2, Arrays.asList("a", "b", "c")) .appendToLog(4, Arrays.asList("d", "e", "f")) .appendToLog(4, Arrays.asList("g", "h", "i")) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); // Start off as the leader and receive a fetch to initialize the high watermark context.becomeLeader(); - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 10L, epoch, 500)); context.client.poll(); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); // Now we receive a vote request which transitions us to the 'voted' state int candidateEpoch = epoch + 1; - context.deliverRequest(context.voteRequest(candidateEpoch, otherNodeId, epoch, 10L)); + context.deliverRequest(context.voteRequest(candidateEpoch, otherNodeKey, epoch, 10L)); context.pollUntilResponse(); - context.assertVotedCandidate(candidateEpoch, otherNodeId); + context.assertVotedCandidate(candidateEpoch, otherNodeKey.id()); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); // Register another listener and verify that it catches up while we remain 'voted' @@ -3150,7 +3479,7 @@ public class KafkaRaftClientTest { ); context.client.register(secondListener); context.client.poll(); - context.assertVotedCandidate(candidateEpoch, otherNodeId); + context.assertVotedCandidate(candidateEpoch, otherNodeKey.id()); // Note the offset is 9 because from offsets 0 to 8 there are data records, // at offset 9 there is a control record and the raft client sends control record to the @@ -3160,34 +3489,36 @@ public class KafkaRaftClientTest { assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } - @Test - public void testHandleCommitCallbackFiresInCandidateState() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) throws Exception { // This test verifies that the state machine can still catch up even while // an election is in progress as long as the high watermark is known. int localId = 0; - int otherNodeId = 1; + ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc); int epoch = 7; - Set voters = Utils.mkSet(localId, otherNodeId); + Set voters = Utils.mkSet(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(2, Arrays.asList("a", "b", "c")) .appendToLog(4, Arrays.asList("d", "e", "f")) .appendToLog(4, Arrays.asList("g", "h", "i")) .withUnknownLeader(epoch - 1) + .withKip853Rpc(withKip853Rpc) .build(); // Start off as the leader and receive a fetch to initialize the high watermark context.becomeLeader(); assertEquals(10L, context.log.endOffset().offset); - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 0)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 10L, epoch, 0)); context.pollUntilResponse(); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); // Now we receive a vote request which transitions us to the 'unattached' state - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch, 9L)); + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 9L)); context.pollUntilResponse(); context.assertUnknownLeader(epoch + 1); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); @@ -3214,8 +3545,11 @@ public class KafkaRaftClientTest { assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } - @Test - public void testHandleLeaderChangeFiresAfterUnattachedRegistration() throws Exception { + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleLeaderChangeFiresAfterUnattachedRegistration( + boolean withKip853Rpc + ) throws Exception { // When registering a listener while the replica is unattached, it should get notified // with the current epoch // When transitioning to follower, expect another notification with the leader and epoch @@ -3227,6 +3561,7 @@ public class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) + .withKip853Rpc(withKip853Rpc) .build(); // Register another listener and verify that it is notified of latest epoch @@ -3331,14 +3666,15 @@ public class KafkaRaftClientTest { } @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testAppendWithRequiredBaseOffset(boolean correctOffset) throws Exception { + @CsvSource({"false,false", "false,true", "true,false", "true,true"}) + public void testAppendWithRequiredBaseOffset(boolean correctOffset, boolean withKip853Rpc) throws Exception { int localId = 0; int otherNodeId = 1; Set voters = Utils.mkSet(localId, otherNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .build(); + .withKip853Rpc(withKip853Rpc) + .build(); context.becomeLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -3357,4 +3693,9 @@ public class KafkaRaftClientTest { }); } } + + private static ReplicaKey replicaKey(int id, boolean withDirectoryId) { + Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; + return ReplicaKey.of(id, directoryId); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java index 1779a231464..866d6573177 100644 --- a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.VoterSet; +import org.apache.kafka.raft.internals.VoterSetTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -31,14 +32,12 @@ import org.mockito.Mockito; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import static java.util.Collections.emptySet; import static java.util.Collections.singleton; @@ -46,12 +45,12 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.raft.LeaderState.CHECK_QUORUM_TIMEOUT_FACTOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class LeaderStateTest { - private final int localId = 0; - private final Uuid localDirectoryId = Uuid.randomUuid(); + private final ReplicaKey localReplicaKey = ReplicaKey.of(0, Uuid.randomUuid()); private final int epoch = 5; private final LogContext logContext = new LogContext(); private final BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); @@ -60,44 +59,75 @@ public class LeaderStateTest { private final int checkQuorumTimeoutMs = (int) (fetchTimeoutMs * CHECK_QUORUM_TIMEOUT_FACTOR); private LeaderState newLeaderState( - Set voters, + VoterSet voters, long epochStartOffset ) { return new LeaderState<>( time, - localId, - Uuid.randomUuid(), + localReplicaKey, epoch, epochStartOffset, - toMap(voters), voters, + voters.voterIds(), accumulator, + voters.listeners(localReplicaKey.id()), fetchTimeoutMs, logContext ); } - @Test - public void testRequireNonNullAccumulator() { - assertThrows(NullPointerException.class, () -> new LeaderState<>( - new MockTime(), - localId, - Uuid.randomUuid(), - epoch, - 0, - Collections.emptyMap(), - Collections.emptySet(), - null, - fetchTimeoutMs, - logContext - )); + private VoterSet localWithRemoteVoterSet(IntStream remoteIds, boolean withDirectoryId) { + Map voters = VoterSetTest.voterMap(remoteIds, withDirectoryId); + if (withDirectoryId) { + voters.put(localReplicaKey.id(), VoterSetTest.voterNode(localReplicaKey)); + } else { + voters.put( + localReplicaKey.id(), + VoterSetTest.voterNode(ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID)) + ); + } + + return VoterSetTest.voterSet(voters); + } + + private VoterSet localWithRemoteVoterSet(Stream remoteReplicaKeys, boolean withDirectoryId) { + ReplicaKey actualLocalVoter = withDirectoryId ? + localReplicaKey : + ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); + + return VoterSetTest.voterSet( + Stream.concat(Stream.of(actualLocalVoter), remoteReplicaKeys) + ); } @Test - public void testFollowerAcknowledgement() { + public void testRequireNonNullAccumulator() { + assertThrows( + NullPointerException.class, + () -> new LeaderState<>( + new MockTime(), + localReplicaKey, + epoch, + 0, + VoterSetTest.voterSet(Stream.of(localReplicaKey)), + Collections.emptySet(), + null, + Endpoints.empty(), + fetchTimeoutMs, + logContext + ) + ); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testFollowerAcknowledgement(boolean withDirectoryId) { int node1 = 1; int node2 = 2; - LeaderState state = newLeaderState(mkSet(localId, node1, node2), 0L); + LeaderState state = newLeaderState( + localWithRemoteVoterSet(IntStream.of(node1, node2), withDirectoryId), + 0L + ); assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters()); state.addAcknowledgementFrom(node1); assertEquals(singleton(node2), state.nonAcknowledgingVoters()); @@ -108,345 +138,444 @@ public class LeaderStateTest { @Test public void testNonFollowerAcknowledgement() { int nonVoterId = 1; - LeaderState state = newLeaderState(singleton(localId), 0L); + LeaderState state = newLeaderState( + VoterSetTest.voterSet(Stream.of(localReplicaKey)), + 0L + ); assertThrows(IllegalArgumentException.class, () -> state.addAcknowledgementFrom(nonVoterId)); } @Test public void testUpdateHighWatermarkQuorumSizeOne() { - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, 15L); + assertEquals(Optional.empty(), state.highWatermark()); - assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), toMap(voterSet))); + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters)); assertEquals(emptySet(), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), toMap(voterSet))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters)); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); - assertTrue(state.updateLocalState(new LogOffsetMetadata(20), toMap(voterSet))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(20), voters)); assertEquals(Optional.of(new LogOffsetMetadata(20L)), state.highWatermark()); } @Test public void testNonMonotonicLocalEndOffsetUpdate() { - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, 15L); + assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), toMap(voterSet))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters)); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); - assertThrows(IllegalStateException.class, - () -> state.updateLocalState(new LogOffsetMetadata(15L), toMap(voterSet))); + assertThrows( + IllegalStateException.class, + () -> state.updateLocalState(new LogOffsetMetadata(15L), voters) + ); } - @Test - public void testLastCaughtUpTimeVoters() { - int node1 = 1; - int node2 = 2; + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testLastCaughtUpTimeVoters(boolean withDirectoryId) { + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); + ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId); int currentTime = 1000; int fetchTime = 0; int caughtUpTime = -1; - Set voterSet = mkSet(localId, node1, node2); - LeaderState state = newLeaderState(voterSet, 10L); + VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId); + LeaderState state = newLeaderState(voters, 10L); + assertEquals(Optional.empty(), state.highWatermark()); - assertFalse(state.updateLocalState(new LogOffsetMetadata(10L), toMap(voterSet))); - assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters()); + assertFalse(state.updateLocalState(new LogOffsetMetadata(10L), voters)); + assertEquals(mkSet(nodeKey1.id(), nodeKey2.id()), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); // Node 1 falls behind - assertFalse(state.updateLocalState(new LogOffsetMetadata(11L), toMap(voterSet))); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(10L))); - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeVoterState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertFalse(state.updateLocalState(new LogOffsetMetadata(11L), voters)); + assertFalse(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(10L))); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeVoterState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 catches up to leader - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(11L))); + assertTrue(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(11L))); caughtUpTime = fetchTime; - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeVoterState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeVoterState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 falls behind - assertFalse(state.updateLocalState(new LogOffsetMetadata(100L), toMap(voterSet))); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(50L))); - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeVoterState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertFalse(state.updateLocalState(new LogOffsetMetadata(100L), voters)); + assertTrue(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(50L))); + assertEquals(currentTime, describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp()); + assertEquals( + caughtUpTime, + describeVoterState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 catches up to the last fetch offset int prevFetchTime = fetchTime; - assertFalse(state.updateLocalState(new LogOffsetMetadata(200L), toMap(voterSet))); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(100L))); + assertFalse(state.updateLocalState(new LogOffsetMetadata(200L), voters)); + assertTrue(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(100L))); caughtUpTime = prevFetchTime; - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeVoterState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeVoterState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node2 has never caught up to leader - assertEquals(-1L, describeVoterState(state, node2, currentTime).lastCaughtUpTimestamp()); - assertFalse(state.updateLocalState(new LogOffsetMetadata(300L), toMap(voterSet))); - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(200L))); - assertEquals(-1L, describeVoterState(state, node2, currentTime).lastCaughtUpTimestamp()); - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(250L))); - assertEquals(-1L, describeVoterState(state, node2, currentTime).lastCaughtUpTimestamp()); + assertEquals( + -1L, + describeVoterState(state, nodeKey2.id(), currentTime).lastCaughtUpTimestamp() + ); + assertFalse(state.updateLocalState(new LogOffsetMetadata(300L), voters)); + assertTrue(state.updateReplicaState(nodeKey2, ++fetchTime, new LogOffsetMetadata(200L))); + assertEquals( + -1L, + describeVoterState(state, nodeKey2.id(), currentTime).lastCaughtUpTimestamp() + ); + assertTrue(state.updateReplicaState(nodeKey2, ++fetchTime, new LogOffsetMetadata(250L))); + assertEquals( + -1L, + describeVoterState(state, nodeKey2.id(), currentTime).lastCaughtUpTimestamp() + ); } - @Test - public void testLastCaughtUpTimeObserver() { - int node1 = 1; + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testLastCaughtUpTimeObserver(boolean withDirectoryId) { + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); int currentTime = 1000; int fetchTime = 0; int caughtUpTime = -1; - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, 5L); + + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, 5L); + assertEquals(Optional.empty(), state.highWatermark()); assertEquals(emptySet(), state.nonAcknowledgingVoters()); // Node 1 falls behind - assertTrue(state.updateLocalState(new LogOffsetMetadata(11L), toMap(voterSet))); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(10L))); - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeObserverState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertTrue(state.updateLocalState(new LogOffsetMetadata(11L), voters)); + assertFalse(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(10L))); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeObserverState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 catches up to leader - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(11L))); + assertFalse(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(11L))); caughtUpTime = fetchTime; - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeObserverState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeObserverState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 falls behind - assertTrue(state.updateLocalState(new LogOffsetMetadata(100L), toMap(voterSet))); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(50L))); - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeObserverState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertTrue(state.updateLocalState(new LogOffsetMetadata(100L), voters)); + assertFalse(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(50L))); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeObserverState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 catches up to the last fetch offset int prevFetchTime = fetchTime; - assertTrue(state.updateLocalState(new LogOffsetMetadata(200L), toMap(voterSet))); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(102L))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(200L), voters)); + assertFalse(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(102L))); caughtUpTime = prevFetchTime; - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeObserverState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeObserverState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); // Node 1 catches up to leader - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), ++fetchTime, new LogOffsetMetadata(200L))); + assertFalse(state.updateReplicaState(nodeKey1, ++fetchTime, new LogOffsetMetadata(200L))); caughtUpTime = fetchTime; - assertEquals(currentTime, describeVoterState(state, localId, currentTime).lastCaughtUpTimestamp()); - assertEquals(caughtUpTime, describeObserverState(state, node1, currentTime).lastCaughtUpTimestamp()); + assertEquals( + currentTime, + describeVoterState(state, localReplicaKey.id(), currentTime).lastCaughtUpTimestamp() + ); + assertEquals( + caughtUpTime, + describeObserverState(state, nodeKey1.id(), currentTime).lastCaughtUpTimestamp() + ); } @Test public void testIdempotentEndOffsetUpdate() { - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, 15L); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), toMap(voterSet))); - assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), toMap(voterSet))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters)); + assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), voters)); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); } @Test public void testUpdateHighWatermarkMetadata() { - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, 15L); assertEquals(Optional.empty(), state.highWatermark()); LogOffsetMetadata initialHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("bar"))); - assertTrue(state.updateLocalState(initialHw, toMap(voterSet))); + assertTrue(state.updateLocalState(initialHw, voters)); assertEquals(Optional.of(initialHw), state.highWatermark()); LogOffsetMetadata updateHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("baz"))); - assertTrue(state.updateLocalState(updateHw, toMap(voterSet))); + assertTrue(state.updateLocalState(updateHw, voters)); assertEquals(Optional.of(updateHw), state.highWatermark()); } - @Test - public void testUpdateHighWatermarkQuorumSizeTwo() { - int otherNodeId = 1; - Set voterSet = mkSet(localId, otherNodeId); - LeaderState state = newLeaderState(voterSet, 10L); - assertFalse(state.updateLocalState(new LogOffsetMetadata(13L), toMap(voterSet))); - assertEquals(singleton(otherNodeId), state.nonAcknowledgingVoters()); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testUpdateHighWatermarkQuorumSizeTwo(boolean withDirectoryId) { + ReplicaKey otherNodeKey = replicaKey(1, withDirectoryId); + + VoterSet voters = localWithRemoteVoterSet(Stream.of(otherNodeKey), withDirectoryId); + LeaderState state = newLeaderState(voters, 10L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(13L), voters)); + assertEquals(singleton(otherNodeKey.id()), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertFalse(state.updateReplicaState(otherNodeId, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + assertFalse(state.updateReplicaState(otherNodeKey, 0, new LogOffsetMetadata(10L))); assertEquals(emptySet(), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateReplicaState(otherNodeId, Uuid.randomUuid(), 0, new LogOffsetMetadata(11L))); + assertTrue(state.updateReplicaState(otherNodeKey, 0, new LogOffsetMetadata(11L))); assertEquals(Optional.of(new LogOffsetMetadata(11L)), state.highWatermark()); - assertTrue(state.updateReplicaState(otherNodeId, Uuid.randomUuid(), 0, new LogOffsetMetadata(13L))); + assertTrue(state.updateReplicaState(otherNodeKey, 0, new LogOffsetMetadata(13L))); assertEquals(Optional.of(new LogOffsetMetadata(13L)), state.highWatermark()); } - @Test - public void testUpdateHighWatermarkQuorumSizeThree() { - int node1 = 1; - int node2 = 2; - Set voterSet = mkSet(localId, node1, node2); - LeaderState state = newLeaderState(voterSet, 10L); - assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), toMap(voterSet))); - assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters()); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testUpdateHighWatermarkQuorumSizeThree(boolean withDirectoryId) { + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); + ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId); + + VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId); + LeaderState state = newLeaderState(voters, 10L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters)); + assertEquals(mkSet(nodeKey1.id(), nodeKey2.id()), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); - assertEquals(singleton(node2), state.nonAcknowledgingVoters()); + assertFalse(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(10L))); + assertEquals(singleton(nodeKey2.id()), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(10L))); assertEquals(emptySet(), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); + assertTrue(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(15L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateLocalState(new LogOffsetMetadata(20L), toMap(voterSet))); + assertFalse(state.updateLocalState(new LogOffsetMetadata(20L), voters)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(20L))); + assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(20L))); assertEquals(Optional.of(new LogOffsetMetadata(20L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(20L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(20L))); assertEquals(Optional.of(new LogOffsetMetadata(20L)), state.highWatermark()); } @Test public void testHighWatermarkDoesIncreaseFromNewVoter() { - int node1 = 1; - int node2 = 2; - Set originalVoterSet = mkSet(localId, node1); - LeaderState state = newLeaderState(originalVoterSet, 5L); - assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), toMap(originalVoterSet))); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); + + VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1), true); + LeaderState state = newLeaderState(originalVoters, 5L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); + assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(10L))); assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); // updating replica state of node2 before it joins voterSet should not increase HW to 15L - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(15L))); assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); // adding node2 to voterSet will cause HW to increase to 15L - Set voterSetWithNode2 = mkSet(localId, node1, node2); - assertTrue(state.updateLocalState(new LogOffsetMetadata(15L), toMap(voterSetWithNode2))); + VoterSet votersWithNode2 = originalVoters.addVoter(VoterSetTest.voterNode(nodeKey2)).get(); + assertTrue(state.updateLocalState(new LogOffsetMetadata(15L), votersWithNode2)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW will not update to 16L until a majority reaches it - assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), toMap(voterSetWithNode2))); + assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), votersWithNode2)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(16L))); + assertTrue(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); } @Test public void testHighWatermarkDoesNotDecreaseFromNewVoter() { - int node1 = 1; - int node2 = 2; - int node3 = 3; + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); + ReplicaKey nodeKey3 = ReplicaKey.of(3, Uuid.randomUuid()); + // start with three voters with HW at 15L - Set originalVoterSet = mkSet(localId, node1, node2); - LeaderState state = newLeaderState(originalVoterSet, 5L); - assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), toMap(originalVoterSet))); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); + VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); + LeaderState state = newLeaderState(originalVoters, 5L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); + assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(10L))); // updating replica state of node3 before it joins voterSet - assertFalse(state.updateReplicaState(node3, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + assertFalse(state.updateReplicaState(nodeKey3, 0, new LogOffsetMetadata(10L))); // adding node3 to voterSet should not cause HW to decrease even if majority is < HW - Set voterSetWithNode3 = mkSet(localId, node1, node2, node3); - assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), toMap(voterSetWithNode3))); + VoterSet votersWithNode3 = originalVoters.addVoter(VoterSetTest.voterNode(nodeKey3)).get(); + assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), votersWithNode3)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW will not decrease if calculated HW is anything lower than the last HW - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(13L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(13L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node3, Uuid.randomUuid(), 0, new LogOffsetMetadata(13L))); + assertFalse(state.updateReplicaState(nodeKey3, 0, new LogOffsetMetadata(13L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(16L))); + assertFalse(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW will update to 16L once a majority of the voterSet is at least 16L - assertTrue(state.updateReplicaState(node3, Uuid.randomUuid(), 0, new LogOffsetMetadata(16L))); + assertTrue(state.updateReplicaState(nodeKey3, 0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); } @Test public void testUpdateHighWatermarkRemovingFollowerFromVoterStates() { - int node1 = 1; - int node2 = 2; - Set originalVoterSet = mkSet(localId, node1, node2); - LeaderState state = newLeaderState(originalVoterSet, 10L); - assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), toMap(originalVoterSet))); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); + + VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); + LeaderState state = newLeaderState(originalVoters, 10L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); + assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(10L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // removing node1 should not decrement HW to 10L - Set voterSetWithoutNode1 = mkSet(localId, node2); - assertFalse(state.updateLocalState(new LogOffsetMetadata(17L), toMap(voterSetWithoutNode1))); + VoterSet votersWithoutNode1 = originalVoters.removeVoter(nodeKey1).get(); + assertFalse(state.updateLocalState(new LogOffsetMetadata(17L), votersWithoutNode1)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW cannot change until after node2 catches up to last HW - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(14L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(14L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateLocalState(new LogOffsetMetadata(18L), toMap(voterSetWithoutNode1))); + assertFalse(state.updateLocalState(new LogOffsetMetadata(18L), votersWithoutNode1)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(18L))); + assertFalse(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(18L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(15L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW should update to 16L - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(16L))); + assertTrue(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); } @Test public void testUpdateHighWatermarkQuorumRemovingLeaderFromVoterStates() { - int node1 = 1; - int node2 = 2; - Set originalVoterSet = mkSet(localId, node1, node2); - LeaderState state = newLeaderState(originalVoterSet, 10L); - assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), toMap(originalVoterSet))); - assertTrue(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(10L))); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); + + VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); + LeaderState state = newLeaderState(originalVoters, 10L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); + assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(10L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // removing leader should not decrement HW to 10L - Set voterSetWithoutLeader = mkSet(node1, node2); - assertFalse(state.updateLocalState(new LogOffsetMetadata(17L), toMap(voterSetWithoutLeader))); + VoterSet votersWithoutLeader = originalVoters.removeVoter(localReplicaKey).get(); + assertFalse(state.updateLocalState(new LogOffsetMetadata(17L), votersWithoutLeader)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW cannot change until node2 catches up to last HW - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), 0, new LogOffsetMetadata(16L))); + assertFalse(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateLocalState(new LogOffsetMetadata(18L), toMap(voterSetWithoutLeader))); + assertFalse(state.updateLocalState(new LogOffsetMetadata(18L), votersWithoutLeader)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(14L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(14L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(15L))); + assertFalse(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(15L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // HW will not update to 16L until majority of remaining voterSet (node1, node2) are at least 16L - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(16L))); + assertTrue(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); } - @Test - public void testNonMonotonicHighWatermarkUpdate() { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testNonMonotonicHighWatermarkUpdate(boolean withDirectoryId) { MockTime time = new MockTime(); - int node1 = 1; - Set voterSet = mkSet(localId, node1); - LeaderState state = newLeaderState(voterSet, 0L); - state.updateLocalState(new LogOffsetMetadata(10L), toMap(voterSet)); - state.updateReplicaState(node1, Uuid.randomUuid(), time.milliseconds(), new LogOffsetMetadata(10L)); + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); + + VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1), withDirectoryId); + LeaderState state = newLeaderState(voters, 0L); + + state.updateLocalState(new LogOffsetMetadata(10L), voters); + state.updateReplicaState(nodeKey1, time.milliseconds(), new LogOffsetMetadata(10L)); assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); // Follower crashes and disk is lost. It fetches an earlier offset to rebuild state. // The leader will report an error in the logs, but will not let the high watermark rewind - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), time.milliseconds(), new LogOffsetMetadata(5L))); - assertEquals(5L, describeVoterState(state, node1, time.milliseconds()).logEndOffset()); + assertFalse(state.updateReplicaState(nodeKey1, time.milliseconds(), new LogOffsetMetadata(5L))); + assertEquals(5L, describeVoterState(state, nodeKey1.id(), time.milliseconds()).logEndOffset()); assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); } - @Test - public void testGetNonLeaderFollowersByFetchOffsetDescending() { - int node1 = 1; - int node2 = 2; + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGetNonLeaderFollowersByFetchOffsetDescending(boolean withDirectoryId) { + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); + ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId); long leaderStartOffset = 10L; long leaderEndOffset = 15L; - LeaderState state = setUpLeaderAndFollowers(node1, node2, leaderStartOffset, leaderEndOffset); + VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId); + LeaderState state = newLeaderState(voters, leaderStartOffset); + + state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), voters); + assertEquals(Optional.empty(), state.highWatermark()); + state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(leaderStartOffset)); + state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(leaderEndOffset)); // Leader should not be included; the follower with larger offset should be prioritized. - assertEquals(Arrays.asList(node2, node1), state.nonLeaderVotersByDescendingFetchOffset()); + assertEquals( + Arrays.asList(nodeKey2.id(), nodeKey1.id()), + state.nonLeaderVotersByDescendingFetchOffset() + ); } @Test @@ -455,224 +584,246 @@ public class LeaderStateTest { long leaderStartOffset = 10L; long leaderEndOffset = 15L; - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, leaderStartOffset); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, leaderStartOffset); // Until we have updated local state, high watermark should be uninitialized assertEquals(Optional.empty(), state.highWatermark()); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); assertEquals(-1, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(epoch, partitionData.leaderEpoch()); assertEquals(Collections.emptyList(), partitionData.observers()); assertEquals(1, partitionData.currentVoters().size()); - assertEquals(new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(localId) - .setReplicaDirectoryId(localDirectoryId) + assertEquals( + new DescribeQuorumResponseData.ReplicaState() + .setReplicaId(localReplicaKey.id()) + .setReplicaDirectoryId(localReplicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(-1) .setLastFetchTimestamp(time.milliseconds()) .setLastCaughtUpTimestamp(time.milliseconds()), - partitionData.currentVoters().get(0)); + partitionData.currentVoters().get(0) + ); // Now update the high watermark and verify the describe output - assertTrue(state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), toMap(voterSet))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), voters)); assertEquals(Optional.of(new LogOffsetMetadata(leaderEndOffset)), state.highWatermark()); time.sleep(500); partitionData = state.describeQuorum(time.milliseconds()); assertEquals(leaderEndOffset, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(epoch, partitionData.leaderEpoch()); assertEquals(Collections.emptyList(), partitionData.observers()); assertEquals(1, partitionData.currentVoters().size()); - assertEquals(new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(localId) - .setReplicaDirectoryId(localDirectoryId) + assertEquals( + new DescribeQuorumResponseData.ReplicaState() + .setReplicaId(localReplicaKey.id()) + .setReplicaDirectoryId(localReplicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(leaderEndOffset) .setLastFetchTimestamp(time.milliseconds()) .setLastCaughtUpTimestamp(time.milliseconds()), - partitionData.currentVoters().get(0)); + partitionData.currentVoters().get(0) + ); } - @Test - public void testDescribeQuorumWithMultipleVoters() { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDescribeQuorumWithMultipleVoters(boolean withDirectoryId) { MockTime time = new MockTime(); - int activeFollowerId = 1; - Uuid activeFollowerDirectoryId = Uuid.randomUuid(); - int inactiveFollowerId = 2; - Uuid inactiveFollowerDirectoryId = Uuid.randomUuid(); + Uuid localVoterDirectoryId = withDirectoryId ? + localReplicaKey.directoryId().get() : + ReplicaKey.NO_DIRECTORY_ID; + ReplicaKey activeFollowerKey = replicaKey(1, withDirectoryId); + ReplicaKey inactiveFollowerKey = replicaKey(2, withDirectoryId); long leaderStartOffset = 10L; long leaderEndOffset = 15L; - Map voters = new HashMap<>(); - voters.put(localId, voterNode(localId, localDirectoryId)); - voters.put(activeFollowerId, voterNode(activeFollowerId, activeFollowerDirectoryId)); - voters.put(inactiveFollowerId, voterNode(inactiveFollowerId, inactiveFollowerDirectoryId)); - - LeaderState state = new LeaderState<>( - time, - localId, - Uuid.randomUuid(), - epoch, - leaderStartOffset, - voters, - voters.keySet(), - accumulator, - fetchTimeoutMs, - logContext + VoterSet voters = localWithRemoteVoterSet( + Stream.of(activeFollowerKey, inactiveFollowerKey), + withDirectoryId ); + LeaderState state = newLeaderState(voters, leaderStartOffset); + assertFalse(state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), voters)); assertEquals(Optional.empty(), state.highWatermark()); long activeFollowerFetchTimeMs = time.milliseconds(); - assertTrue(state.updateReplicaState(activeFollowerId, activeFollowerDirectoryId, activeFollowerFetchTimeMs, new LogOffsetMetadata(leaderEndOffset))); + assertTrue( + state.updateReplicaState( + activeFollowerKey, + activeFollowerFetchTimeMs, + new LogOffsetMetadata(leaderEndOffset) + ) + ); assertEquals(Optional.of(new LogOffsetMetadata(leaderEndOffset)), state.highWatermark()); time.sleep(500); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); assertEquals(leaderEndOffset, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(epoch, partitionData.leaderEpoch()); assertEquals(Collections.emptyList(), partitionData.observers()); List voterStates = partitionData.currentVoters(); assertEquals(3, voterStates.size()); - DescribeQuorumResponseData.ReplicaState leaderState = - findReplicaOrFail(localId, partitionData.currentVoters()); - assertEquals(new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(localId) - .setReplicaDirectoryId(localDirectoryId) + DescribeQuorumResponseData.ReplicaState leaderState = findReplicaOrFail( + localReplicaKey.id(), + partitionData.currentVoters() + ); + assertEquals( + new DescribeQuorumResponseData.ReplicaState() + .setReplicaId(localReplicaKey.id()) + .setReplicaDirectoryId(localVoterDirectoryId) .setLogEndOffset(leaderEndOffset) .setLastFetchTimestamp(time.milliseconds()) .setLastCaughtUpTimestamp(time.milliseconds()), - leaderState); + leaderState + ); - DescribeQuorumResponseData.ReplicaState activeFollowerState = - findReplicaOrFail(activeFollowerId, partitionData.currentVoters()); - assertEquals(new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(activeFollowerId) - .setReplicaDirectoryId(activeFollowerDirectoryId) + DescribeQuorumResponseData.ReplicaState activeFollowerState = findReplicaOrFail( + activeFollowerKey.id(), + partitionData.currentVoters() + ); + assertEquals( + new DescribeQuorumResponseData.ReplicaState() + .setReplicaId(activeFollowerKey.id()) + .setReplicaDirectoryId(activeFollowerKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(leaderEndOffset) .setLastFetchTimestamp(activeFollowerFetchTimeMs) .setLastCaughtUpTimestamp(activeFollowerFetchTimeMs), - activeFollowerState); + activeFollowerState + ); - DescribeQuorumResponseData.ReplicaState inactiveFollowerState = - findReplicaOrFail(inactiveFollowerId, partitionData.currentVoters()); - assertEquals(new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(inactiveFollowerId) - .setReplicaDirectoryId(inactiveFollowerDirectoryId) + DescribeQuorumResponseData.ReplicaState inactiveFollowerState = findReplicaOrFail( + inactiveFollowerKey.id(), + partitionData.currentVoters() + ); + assertEquals( + new DescribeQuorumResponseData.ReplicaState() + .setReplicaId(inactiveFollowerKey.id()) + .setReplicaDirectoryId(inactiveFollowerKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(-1) .setLastFetchTimestamp(-1) .setLastCaughtUpTimestamp(-1), - inactiveFollowerState); + inactiveFollowerState + ); } - private LeaderState setUpLeaderAndFollowers(int follower1, - int follower2, - long leaderStartOffset, - long leaderEndOffset) { - Set voterSet = mkSet(localId, follower1, follower2); - LeaderState state = newLeaderState(voterSet, leaderStartOffset); - state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), toMap(voterSet)); - assertEquals(Optional.empty(), state.highWatermark()); - state.updateReplicaState(follower1, Uuid.randomUuid(), 0, new LogOffsetMetadata(leaderStartOffset)); - state.updateReplicaState(follower2, Uuid.randomUuid(), 0, new LogOffsetMetadata(leaderEndOffset)); - return state; - } - - @Test - public void testDescribeQuorumWithObservers() { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDescribeQuorumWithObservers(boolean withDirectoryId) { MockTime time = new MockTime(); - int observerId = 10; - Uuid observerDirectoryId = Uuid.randomUuid(); + Uuid localVoterDirectoryId = withDirectoryId ? + localReplicaKey.directoryId().get() : + ReplicaKey.NO_DIRECTORY_ID; + + ReplicaKey observerKey = replicaKey(10, withDirectoryId); long epochStartOffset = 10L; - Set voterSet = singleton(localId); - LeaderState state = newLeaderState(voterSet, epochStartOffset); - assertTrue(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 1), toMap(voterSet))); + VoterSet voters = localWithRemoteVoterSet(Stream.empty(), withDirectoryId); + LeaderState state = newLeaderState(voters, epochStartOffset); + assertTrue(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 1), voters)); assertEquals(Optional.of(new LogOffsetMetadata(epochStartOffset + 1)), state.highWatermark()); time.sleep(500); long observerFetchTimeMs = time.milliseconds(); - assertFalse(state.updateReplicaState(observerId, observerDirectoryId, observerFetchTimeMs, new LogOffsetMetadata(epochStartOffset + 1))); + assertFalse( + state.updateReplicaState( + observerKey, + observerFetchTimeMs, + new LogOffsetMetadata(epochStartOffset + 1) + ) + ); time.sleep(500); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); assertEquals(epochStartOffset + 1, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(epoch, partitionData.leaderEpoch()); assertEquals(1, partitionData.currentVoters().size()); - assertEquals(localId, partitionData.currentVoters().get(0).replicaId()); + assertEquals(localReplicaKey.id(), partitionData.currentVoters().get(0).replicaId()); + assertEquals( + localVoterDirectoryId, + partitionData.currentVoters().get(0).replicaDirectoryId() + ); List observerStates = partitionData.observers(); assertEquals(1, observerStates.size()); DescribeQuorumResponseData.ReplicaState observerState = observerStates.get(0); assertEquals(new DescribeQuorumResponseData.ReplicaState() - .setReplicaId(observerId) - .setReplicaDirectoryId(observerDirectoryId) + .setReplicaId(observerKey.id()) + .setReplicaDirectoryId(observerKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) .setLogEndOffset(epochStartOffset + 1) .setLastFetchTimestamp(observerFetchTimeMs) .setLastCaughtUpTimestamp(observerFetchTimeMs), observerState); } - @Test - public void testDescribeQuorumWithVotersAndObservers() { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDescribeQuorumWithVotersAndObservers(boolean withDirectoryId) { MockTime time = new MockTime(); - int leader = localId; - int node1 = 1; - int node2 = 2; + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); + ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId); long epochStartOffset = 10L; - Set voterSet = mkSet(leader, node1, node2); - LeaderState state = newLeaderState(voterSet, epochStartOffset); - assertFalse(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 1), toMap(voterSet))); - assertTrue(state.updateReplicaState(node2, Uuid.randomUuid(), 0, new LogOffsetMetadata(epochStartOffset + 1))); + VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId); + LeaderState state = newLeaderState(voters, epochStartOffset); + + + assertFalse(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 1), voters)); + assertTrue(state.updateReplicaState(nodeKey2, 0, new LogOffsetMetadata(epochStartOffset + 1))); assertEquals(Optional.of(new LogOffsetMetadata(epochStartOffset + 1)), state.highWatermark()); // node1 becomes an observer long fetchTimeMs = time.milliseconds(); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), fetchTimeMs, new LogOffsetMetadata(epochStartOffset + 1))); - Set voterSetWithoutNode1 = mkSet(leader, node2); - state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 5), toMap(voterSetWithoutNode1)); - + assertFalse(state.updateReplicaState(nodeKey1, fetchTimeMs, new LogOffsetMetadata(epochStartOffset + 1))); + VoterSet votersWithoutNode1 = voters.removeVoter(nodeKey1).get(); + state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 5), votersWithoutNode1); time.sleep(500); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); assertEquals(epochStartOffset + 1, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(epoch, partitionData.leaderEpoch()); DescribeQuorumResponseData.ReplicaState observer = partitionData.observers().get(0); - assertEquals(node1, observer.replicaId()); + assertEquals(nodeKey1.id(), observer.replicaId()); + assertEquals( + nodeKey1.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID), + observer.replicaDirectoryId() + ); assertEquals(epochStartOffset + 1, observer.logEndOffset()); assertEquals(2, partitionData.currentVoters().size()); // node1 catches up with leader, HW should not change time.sleep(500); fetchTimeMs = time.milliseconds(); - assertFalse(state.updateReplicaState(node1, Uuid.randomUuid(), fetchTimeMs, new LogOffsetMetadata(epochStartOffset + 5))); + assertFalse(state.updateReplicaState(nodeKey1, fetchTimeMs, new LogOffsetMetadata(epochStartOffset + 5))); assertEquals(Optional.of(new LogOffsetMetadata(epochStartOffset + 1)), state.highWatermark()); // node1 becomes a voter again, HW should change - assertTrue(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 10), toMap(voterSet))); + assertTrue(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 10), voters)); time.sleep(500); partitionData = state.describeQuorum(time.milliseconds()); assertEquals(epochStartOffset + 5, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(epoch, partitionData.leaderEpoch()); assertEquals(Collections.emptyList(), partitionData.observers()); assertEquals(3, partitionData.currentVoters().size()); - DescribeQuorumResponseData.ReplicaState node1State = partitionData.currentVoters().stream() - .filter(replicaState -> replicaState.replicaId() == node1) - .findFirst().get(); + DescribeQuorumResponseData.ReplicaState node1State = partitionData + .currentVoters() + .stream() + .filter(replicaState -> replicaState.replicaId() == nodeKey1.id()) + .findFirst() + .get(); assertEquals(epochStartOffset + 5, node1State.logEndOffset()); assertEquals(fetchTimeMs, node1State.lastFetchTimestamp()); } @@ -680,61 +831,69 @@ public class LeaderStateTest { @Test public void testClearInactiveObserversIgnoresLeader() { MockTime time = new MockTime(); - int followerId = 1; - int observerId = 10; + ReplicaKey followerKey = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey observerKey = ReplicaKey.of(10, Uuid.randomUuid()); long epochStartOffset = 10L; - Set voterSet = mkSet(localId, followerId); - LeaderState state = newLeaderState(voterSet, epochStartOffset); - assertFalse(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 1), toMap(voterSet))); - assertTrue(state.updateReplicaState(followerId, Uuid.randomUuid(), time.milliseconds(), new LogOffsetMetadata(epochStartOffset + 1))); + VoterSet voters = localWithRemoteVoterSet(Stream.of(followerKey), true); + LeaderState state = newLeaderState(voters, epochStartOffset); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 1), voters)); + assertTrue(state.updateReplicaState(followerKey, time.milliseconds(), new LogOffsetMetadata(epochStartOffset + 1))); // observer is returned since its lastFetchTimestamp is within OBSERVER_SESSION_TIMEOUT_MS time.sleep(500); long observerFetchTimeMs = time.milliseconds(); - assertFalse(state.updateReplicaState(observerId, Uuid.randomUuid(), observerFetchTimeMs, new LogOffsetMetadata(epochStartOffset + 1))); + assertFalse(state.updateReplicaState(observerKey, observerFetchTimeMs, new LogOffsetMetadata(epochStartOffset + 1))); time.sleep(500); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); assertEquals(epochStartOffset + 1, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(2, partitionData.currentVoters().size()); assertEquals(1, partitionData.observers().size()); - assertEquals(observerId, partitionData.observers().get(0).replicaId()); + assertEquals(observerKey.id(), partitionData.observers().get(0).replicaId()); // observer is not returned once its lastFetchTimestamp surpasses OBSERVER_SESSION_TIMEOUT_MS time.sleep(LeaderState.OBSERVER_SESSION_TIMEOUT_MS); partitionData = state.describeQuorum(time.milliseconds()); assertEquals(epochStartOffset + 1, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(2, partitionData.currentVoters().size()); assertEquals(0, partitionData.observers().size()); // leader becomes observer - Set voterSetWithoutLeader = mkSet(followerId); - assertFalse(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 10), toMap(voterSetWithoutLeader))); + VoterSet votersWithoutLeader = voters.removeVoter(localReplicaKey).get(); + assertFalse(state.updateLocalState(new LogOffsetMetadata(epochStartOffset + 10), votersWithoutLeader)); // leader should be returned in describe quorum output time.sleep(LeaderState.OBSERVER_SESSION_TIMEOUT_MS); long describeQuorumCalledTime = time.milliseconds(); partitionData = state.describeQuorum(describeQuorumCalledTime); assertEquals(epochStartOffset + 1, partitionData.highWatermark()); - assertEquals(localId, partitionData.leaderId()); + assertEquals(localReplicaKey.id(), partitionData.leaderId()); assertEquals(1, partitionData.currentVoters().size()); assertEquals(1, partitionData.observers().size()); DescribeQuorumResponseData.ReplicaState observer = partitionData.observers().get(0); - assertEquals(localId, observer.replicaId()); + assertEquals(localReplicaKey.id(), observer.replicaId()); assertEquals(describeQuorumCalledTime, observer.lastFetchTimestamp()); } - @Test - public void testCheckQuorum() { - int node1 = 1; - int node2 = 2; - int node3 = 3; - int node4 = 4; - int observer5 = 5; - LeaderState state = newLeaderState(mkSet(localId, node1, node2, node3, node4), 0L); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testCheckQuorum(boolean withDirectoryId) { + ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); + ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId); + ReplicaKey nodeKey3 = replicaKey(3, withDirectoryId); + ReplicaKey nodeKey4 = replicaKey(4, withDirectoryId); + ReplicaKey observerKey5 = replicaKey(5, withDirectoryId); + + VoterSet voters = localWithRemoteVoterSet( + Stream.of(nodeKey1, nodeKey2, nodeKey3, nodeKey4), + withDirectoryId + ); + LeaderState state = newLeaderState(voters, 0L); + assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); int resignLeadershipTimeout = checkQuorumTimeoutMs; @@ -743,8 +902,8 @@ public class LeaderStateTest { assertTrue(state.timeUntilCheckQuorumExpires(time.milliseconds()) > 0); // received fetch requests from 2 voter nodes, the timer should be reset - state.updateCheckQuorumForFollowingVoter(node1, time.milliseconds()); - state.updateCheckQuorumForFollowingVoter(node2, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey1, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey2, time.milliseconds()); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // Since the timer was reset, it won't expire this time. @@ -753,8 +912,8 @@ public class LeaderStateTest { assertTrue(remainingMs > 0); // received fetch requests from 1 voter and 1 observer nodes, the timer should not be reset. - state.updateCheckQuorumForFollowingVoter(node3, time.milliseconds()); - state.updateCheckQuorumForFollowingVoter(observer5, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey3, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(observerKey5, time.milliseconds()); assertEquals(remainingMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // This time, the checkQuorum timer will be expired @@ -764,11 +923,12 @@ public class LeaderStateTest { @Test public void testCheckQuorumAfterVoterSetChanges() { - int node1 = 1; - int node2 = 2; - int node3 = 3; - Set originalVoterSet = mkSet(localId, node1, node2); - LeaderState state = newLeaderState(originalVoterSet, 0L); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); + ReplicaKey nodeKey3 = ReplicaKey.of(3, Uuid.randomUuid()); + + VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); + LeaderState state = newLeaderState(originalVoters, 0L); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // checkQuorum timeout not exceeded, should not expire the timer @@ -776,41 +936,45 @@ public class LeaderStateTest { assertEquals(checkQuorumTimeoutMs / 2, state.timeUntilCheckQuorumExpires(time.milliseconds())); // received fetch request from 1 voter node, the timer should be reset - state.updateCheckQuorumForFollowingVoter(node1, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey1, time.milliseconds()); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // Adding 1 new voter to the voter set - Set voterSetWithNode3 = mkSet(localId, node1, node2, node3); - state.updateLocalState(new LogOffsetMetadata(1L), toMap(voterSetWithNode3)); + VoterSet votersWithNode3 = originalVoters.addVoter(VoterSetTest.voterNode(nodeKey3)).get(); + state.updateLocalState(new LogOffsetMetadata(1L), votersWithNode3); time.sleep(checkQuorumTimeoutMs / 2); // received fetch request from 1 voter node, the timer should not be reset because the majority should be 3 - state.updateCheckQuorumForFollowingVoter(node1, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey1, time.milliseconds()); assertEquals(checkQuorumTimeoutMs / 2, state.timeUntilCheckQuorumExpires(time.milliseconds())); // Timer should be reset after receiving another voter's fetch request - state.updateCheckQuorumForFollowingVoter(node2, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey2, time.milliseconds()); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // removing leader from the voter set - Set voterSetWithoutLeader = mkSet(node1, node2, node3); - state.updateLocalState(new LogOffsetMetadata(1L), toMap(voterSetWithoutLeader)); + VoterSet votersWithoutLeader = votersWithNode3.removeVoter(localReplicaKey).get(); + state.updateLocalState(new LogOffsetMetadata(1L), votersWithoutLeader); time.sleep(checkQuorumTimeoutMs / 2); // received fetch request from 1 voter, the timer should not be reset. - state.updateCheckQuorumForFollowingVoter(node2, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey2, time.milliseconds()); assertEquals(checkQuorumTimeoutMs / 2, state.timeUntilCheckQuorumExpires(time.milliseconds())); // received fetch request from another voter, the timer should be reset since the current quorum majority is 2. - state.updateCheckQuorumForFollowingVoter(node1, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter(nodeKey1, time.milliseconds()); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); } @Test public void testCheckQuorumWithOneVoter() { int observer = 1; + // Only 1 voter quorum - LeaderState state = newLeaderState(mkSet(localId), 0L); + LeaderState state = newLeaderState( + VoterSetTest.voterSet(Stream.of(localReplicaKey)), + 0L + ); assertEquals(Long.MAX_VALUE, state.timeUntilCheckQuorumExpires(time.milliseconds())); // When checkQuorum timeout not exceeded and got no fetch request from voter, it should not expire the timer @@ -818,38 +982,97 @@ public class LeaderStateTest { assertEquals(Long.MAX_VALUE, state.timeUntilCheckQuorumExpires(time.milliseconds())); // received fetch requests from 1 observer node, the timer still return Long.MAX_VALUE. - state.updateCheckQuorumForFollowingVoter(observer, time.milliseconds()); + state.updateCheckQuorumForFollowingVoter( + ReplicaKey.of(observer, ReplicaKey.NO_DIRECTORY_ID), + time.milliseconds() + ); assertEquals(Long.MAX_VALUE, state.timeUntilCheckQuorumExpires(time.milliseconds())); } + @Test + public void testLeaderEndpoints() { + VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + LeaderState state = newLeaderState(voters, 0L); + + assertNotEquals(Endpoints.empty(), state.leaderEndpoints()); + assertEquals(voters.listeners(localReplicaKey.id()), state.leaderEndpoints()); + } + + @Test + public void testUpdateVotersFromNoDirectoryIdToDirectoryId() { + int node1 = 1; + int node2 = 2; + ReplicaKey nodeKey1 = ReplicaKey.of(node1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(node2, Uuid.randomUuid()); + + VoterSet votersBeforeUpgrade = localWithRemoteVoterSet( + IntStream.of(node1, node2), + false + ); + + LeaderState state = newLeaderState(votersBeforeUpgrade, 0L); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(10L), votersBeforeUpgrade)); + assertTrue(state.updateReplicaState(nodeKey1, 0L, new LogOffsetMetadata(10L))); + assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); + + VoterSet votersAfterUpgrade = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); + + assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), votersAfterUpgrade)); + assertTrue(state.updateReplicaState(nodeKey2, 0L, new LogOffsetMetadata(13L))); + assertEquals(Optional.of(new LogOffsetMetadata(13L)), state.highWatermark()); + } + @Test public void testNoOpForNegativeRemoteNodeId() { MockTime time = new MockTime(); int replicaId = -1; long epochStartOffset = 10L; - LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); - assertFalse(state.updateReplicaState(replicaId, Uuid.randomUuid(), 0, new LogOffsetMetadata(epochStartOffset))); + LeaderState state = newLeaderState( + VoterSetTest.voterSet(Stream.of(localReplicaKey)), + epochStartOffset + ); + assertFalse( + state.updateReplicaState( + ReplicaKey.of(replicaId, ReplicaKey.NO_DIRECTORY_ID), + 0, + new LogOffsetMetadata(epochStartOffset) + ) + ); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); List observerStates = partitionData.observers(); assertEquals(Collections.emptyList(), observerStates); } - @Test - public void testObserverStateExpiration() { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testObserverStateExpiration(boolean withDirectoryId) { MockTime time = new MockTime(); - int observerId = 10; + ReplicaKey observerKey = replicaKey(10, withDirectoryId); long epochStartOffset = 10L; - LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); - state.updateReplicaState(observerId, Uuid.randomUuid(), time.milliseconds(), new LogOffsetMetadata(epochStartOffset)); + LeaderState state = newLeaderState( + VoterSetTest.voterSet(Stream.of(localReplicaKey)), + epochStartOffset + ); + + state.updateReplicaState( + observerKey, + time.milliseconds(), + new LogOffsetMetadata(epochStartOffset) + ); DescribeQuorumResponseData.PartitionData partitionData = state.describeQuorum(time.milliseconds()); List observerStates = partitionData.observers(); assertEquals(1, observerStates.size()); DescribeQuorumResponseData.ReplicaState observerState = observerStates.get(0); - assertEquals(observerId, observerState.replicaId()); + assertEquals(observerKey.id(), observerState.replicaId()); + assertEquals( + observerKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID), + observerState.replicaDirectoryId() + ); time.sleep(LeaderState.OBSERVER_SESSION_TIMEOUT_MS); partitionData = state.describeQuorum(time.milliseconds()); @@ -859,11 +1082,20 @@ public class LeaderStateTest { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testGrantVote(boolean isLogUpToDate) { - LeaderState state = newLeaderState(mkSet(1, 2, 3), 1); + LeaderState state = newLeaderState( + VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), false)), + 1 + ); - assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); + assertFalse( + state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse( + state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse( + state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); } private static class MockOffsetMetadata implements OffsetMetadata { @@ -917,11 +1149,8 @@ public class LeaderStateTest { )); } - private Map toMap(Set data) { - return data.stream().collect(Collectors.toMap(Function.identity(), id -> voterNode(id, id == localId ? localDirectoryId : Uuid.randomUuid()))); - } - - private VoterSet.VoterNode voterNode(int id, Uuid directoryId) { - return new VoterSet.VoterNode(ReplicaKey.of(id, Optional.of(directoryId)), null, null); + private ReplicaKey replicaKey(int id, boolean withDirectoryId) { + Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; + return ReplicaKey.of(id, directoryId); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 3438ecf50a2..f02a2d29d6d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.raft; -import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -31,6 +30,7 @@ import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import java.io.UncheckedIOException; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -48,10 +48,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class QuorumStateTest { private final int localId = 0; private final Uuid localDirectoryId = Uuid.randomUuid(); - private final ReplicaKey localVoterKey = ReplicaKey.of( - localId, - Optional.of(localDirectoryId) - ); + private final ReplicaKey localVoterKey = ReplicaKey.of(localId, localDirectoryId); private final int logEndEpoch = 0; private final MockQuorumStateStore store = new MockQuorumStateStore(); private final MockTime time = new MockTime(); @@ -68,9 +65,9 @@ public class QuorumStateTest { return new QuorumState( localId, localDirectoryId, - VoterSetTest.DEFAULT_LISTENER_NAME, () -> voterSet, () -> kraftVersion, + localId.isPresent() ? voterSet.listeners(localId.getAsInt()) : Endpoints.empty(), electionTimeoutMs, fetchTimeoutMs, store, @@ -100,7 +97,7 @@ public class QuorumStateTest { return replicaKey; } - return ReplicaKey.of(replicaKey.id(), Optional.empty()); + return ReplicaKey.of(replicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); } private VoterSet localStandaloneVoterSet() { @@ -112,7 +109,11 @@ public class QuorumStateTest { private VoterSet localWithRemoteVoterSet(IntStream remoteIds, short kraftVersion) { boolean withDirectoryId = kraftVersion > 0; Map voters = VoterSetTest.voterMap(remoteIds, withDirectoryId); - voters.put(localId, VoterSetTest.voterNode(localVoterKey)); + if (withDirectoryId) { + voters.put(localId, VoterSetTest.voterNode(localVoterKey)); + } else { + voters.put(localId, VoterSetTest.voterNode(ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))); + } return VoterSetTest.voterSet(voters); } @@ -170,15 +171,15 @@ public class QuorumStateTest { FollowerState followerState = state.followerStateOrThrow(); assertEquals(epoch, followerState.epoch()); - assertEquals(node1, followerState.leader().id()); + assertEquals(node1, followerState.leaderId()); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testInitializeAsVoted(short kraftVersion) { - ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); - ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); int epoch = 5; VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); @@ -379,7 +380,7 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); - state.transitionToFollower(5, voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get()); + state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); assertEquals(5, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( @@ -420,7 +421,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testCandidateToVoted(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -447,7 +448,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testCandidateToAnyStateLowerEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -459,7 +460,8 @@ public class QuorumStateTest { IllegalStateException.class, () -> state.transitionToFollower( 4, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ) ); assertEquals(6, state.epoch()); @@ -546,7 +548,7 @@ public class QuorumStateTest { state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); - state.transitionToFollower(5, voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get()); + state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); assertEquals(5, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); @@ -589,7 +591,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToVoted(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -619,7 +621,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToAnyStateLowerEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -633,7 +635,8 @@ public class QuorumStateTest { IllegalStateException.class, () -> state.transitionToFollower( 4, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ) ); assertEquals(6, state.epoch()); @@ -660,7 +663,8 @@ public class QuorumStateTest { IllegalStateException.class, () -> state.transitionToFollower( 0, - voters.voterNode(localId, VoterSetTest.DEFAULT_LISTENER_NAME).get() + localId, + voters.listeners(localId) ) ); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(0, localVoterKey)); @@ -669,7 +673,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToLeaderOrResigned(short kraftVersion) { - ReplicaKey leaderKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid()); int epoch = 5; VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey)); store.writeElectionState( @@ -686,7 +690,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToVotedSameEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -719,7 +723,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToVotedHigherEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -745,7 +749,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToCandidate(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -765,7 +769,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToUnattached(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -786,7 +790,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToFollowerSameEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -794,14 +798,15 @@ public class QuorumStateTest { state.transitionToFollower( 5, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(5, followerState.epoch()); assertEquals( - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME), - Optional.of(followerState.leader()) + voters.listeners(otherNodeKey.id()), + followerState.leaderEndpoints() ); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @@ -809,7 +814,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToFollowerHigherEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -817,14 +822,15 @@ public class QuorumStateTest { state.transitionToFollower( 8, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(8, followerState.epoch()); assertEquals( - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME), - Optional.of(followerState.leader()) + voters.listeners(otherNodeKey.id()), + followerState.leaderEndpoints() ); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @@ -832,7 +838,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToAnyStateLowerEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -843,7 +849,8 @@ public class QuorumStateTest { IllegalStateException.class, () -> state.transitionToFollower( 4, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ) ); assertEquals(5, state.epoch()); @@ -866,7 +873,7 @@ public class QuorumStateTest { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); + state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -879,7 +886,7 @@ public class QuorumStateTest { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); + state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); @@ -900,14 +907,14 @@ public class QuorumStateTest { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - state.transitionToVoted(8, ReplicaKey.of(node1, Optional.of(Uuid.randomUuid()))); + state.transitionToVoted(8, ReplicaKey.of(node1, Uuid.randomUuid())); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(8, ReplicaKey.of(node1, Optional.empty())) + () -> state.transitionToVoted(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(8, ReplicaKey.of(node2, Optional.empty())) + () -> state.transitionToVoted(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) ); } @@ -919,17 +926,18 @@ public class QuorumStateTest { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); + state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); state.transitionToFollower( 5, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(5, followerState.epoch()); assertEquals( - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME), - Optional.of(followerState.leader()) + voters.listeners(node2), + followerState.leaderEndpoints() ); assertEquals( Optional.of( @@ -951,17 +959,18 @@ public class QuorumStateTest { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); + state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(8, followerState.epoch()); assertEquals( - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get(), - followerState.leader() + voters.listeners(node2), + followerState.leaderEndpoints() ); assertEquals( Optional.of( @@ -983,7 +992,7 @@ public class QuorumStateTest { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); + state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5)); } @@ -994,7 +1003,7 @@ public class QuorumStateTest { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToVoted(5, ReplicaKey.of(otherNodeId, Optional.empty())); + state.transitionToVoted(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)); long remainingElectionTimeMs = state.votedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); time.sleep(1000); @@ -1011,7 +1020,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testVotedToAnyStateLowerEpoch(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -1022,7 +1031,8 @@ public class QuorumStateTest { IllegalStateException.class, () -> state.transitionToFollower( 4, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ) ); assertEquals(5, state.epoch()); @@ -1048,28 +1058,31 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( 8, - voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node1, + voters.listeners(node1) ) ); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ) ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(8, followerState.epoch()); assertEquals( - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME), - Optional.of(followerState.leader()) + voters.listeners(node2), + followerState.leaderEndpoints() ); assertEquals( Optional.of( @@ -1093,18 +1106,20 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); state.transitionToFollower( 9, - voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node1, + voters.listeners(node1) ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(9, followerState.epoch()); assertEquals( - voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME), - Optional.of(followerState.leader()) + voters.listeners(node1), + followerState.leaderEndpoints() ); assertEquals( Optional.of( @@ -1128,7 +1143,8 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); @@ -1144,7 +1160,8 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); int jitterMs = 2500; @@ -1167,7 +1184,8 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); } @@ -1182,7 +1200,8 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); int jitterMs = 2500; @@ -1205,35 +1224,37 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node2, + voters.listeners(node2) ); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(8, ReplicaKey.of(node1, Optional.empty())) + () -> state.transitionToVoted(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(8, ReplicaKey.of(localId, Optional.empty())) + () -> state.transitionToVoted(8, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(8, ReplicaKey.of(node2, Optional.empty())) + () -> state.transitionToVoted(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) ); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testFollowerToVotedHigherEpoch(short kraftVersion) { - ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); - ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 8, - voters.voterNode(nodeKey2.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + nodeKey2.id(), + voters.listeners(nodeKey2.id()) ); int jitterMs = 2500; @@ -1259,18 +1280,20 @@ public class QuorumStateTest { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 5, - voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeId, + voters.listeners(otherNodeId) ); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, Optional.empty())) + () -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( 4, - voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeId, + voters.listeners(otherNodeId) ) ); assertEquals(5, state.epoch()); @@ -1290,7 +1313,7 @@ public class QuorumStateTest { @ValueSource(shorts = {0, 1}) public void testCanBecomeFollowerOfNonVoter(short kraftVersion) { int otherNodeId = 1; - ReplicaKey nonVoterKey = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); + ReplicaKey nonVoterKey = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -1304,8 +1327,16 @@ public class QuorumStateTest { assertEquals(nonVoterKey, votedState.votedKey()); // Transition to follower - Node nonVoterNode = new Node(nonVoterKey.id(), "non-voter-host", 1234); - state.transitionToFollower(4, nonVoterNode); + state.transitionToFollower( + 4, + nonVoterKey.id(), + Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("non-voter-host", 1234) + ) + ) + ); assertEquals( new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), state.leaderAndEpoch() @@ -1330,7 +1361,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testObserverWithIdCanVote(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -1360,7 +1391,8 @@ public class QuorumStateTest { state.transitionToFollower( 2, - voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() + node1, + voters.listeners(node1) ); state.transitionToUnattached(3); assertTrue(state.isUnattached()); @@ -1385,13 +1417,13 @@ public class QuorumStateTest { assertTrue(state.isObserver()); state.transitionToUnattached(2); - state.transitionToFollower(3, voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); + state.transitionToFollower(3, node1, voters.listeners(node1)); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(3, followerState.epoch()); assertEquals( - voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME), - Optional.of(followerState.leader()) + voters.listeners(node1), + followerState.leaderEndpoints() ); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @@ -1418,7 +1450,7 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testHasRemoteLeader(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -1439,7 +1471,8 @@ public class QuorumStateTest { state.transitionToFollower( state.epoch() + 1, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ); assertTrue(state.hasRemoteLeader()); } @@ -1447,13 +1480,14 @@ public class QuorumStateTest { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testHighWatermarkRetained(short kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.transitionToFollower( 5, - voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) ); FollowerState followerState = state.followerStateOrThrow(); @@ -1495,11 +1529,11 @@ public class QuorumStateTest { assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows( IllegalStateException.class, - () -> state.transitionToVoted(1, ReplicaKey.of(1, Optional.empty())) + () -> state.transitionToVoted(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - state.transitionToFollower(1, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); + state.transitionToFollower(1, 1, voters.listeners(1)); assertTrue(state.isFollower()); state.transitionToUnattached(2); @@ -1519,7 +1553,7 @@ public class QuorumStateTest { store.writeElectionState( ElectionState.withVotedCandidate( epoch, - ReplicaKey.of(votedId, Optional.empty()), + ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), voters.voterIds() ), kraftVersion diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 1ebc875d1d4..50d8afebb58 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -23,12 +23,14 @@ import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; +import org.apache.kafka.common.message.DescribeQuorumRequestData; import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; @@ -43,20 +45,15 @@ import org.apache.kafka.common.record.ControlRecordUtils; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; -import org.apache.kafka.common.requests.BeginQuorumEpochRequest; -import org.apache.kafka.common.requests.BeginQuorumEpochResponse; import org.apache.kafka.common.requests.DescribeQuorumResponse; -import org.apache.kafka.common.requests.EndQuorumEpochRequest; -import org.apache.kafka.common.requests.EndQuorumEpochResponse; import org.apache.kafka.common.requests.FetchSnapshotResponse; -import org.apache.kafka.common.requests.VoteRequest; -import org.apache.kafka.common.requests.VoteResponse; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchBuilder; import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.SnapshotReader; @@ -79,6 +76,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -108,6 +106,7 @@ public final class RaftClientTestContext { private final QuorumStateStore quorumStateStore; final Uuid clusterId; private final OptionalInt localId; + public final Uuid localDirectoryId; public final KafkaRaftClient client; final Metrics metrics; public final MockLog log; @@ -115,8 +114,10 @@ public final class RaftClientTestContext { final MockMessageQueue messageQueue; final MockTime time; final MockListener listener; - final Set voters; + final VoterSet voters; final Set bootstrapIds; + // Used to determine which RPC request and response to construct + final boolean kip853Rpc; private final List sentResponses = new ArrayList<>(); @@ -142,7 +143,6 @@ public final class RaftClientTestContext { private final Uuid clusterId = Uuid.randomUuid(); private final Set voters; private final OptionalInt localId; - private final Uuid localDirectoryId = Uuid.randomUuid(); private final short kraftVersion = 0; private int requestTimeoutMs = DEFAULT_REQUEST_TIMEOUT_MS; @@ -150,6 +150,7 @@ public final class RaftClientTestContext { private int appendLingerMs = DEFAULT_APPEND_LINGER_MS; private MemoryPool memoryPool = MemoryPool.NONE; private List bootstrapServers = Collections.emptyList(); + private boolean kip853Rpc = false; public Builder(int localId, Set voters) { this(OptionalInt.of(localId), voters); @@ -249,6 +250,11 @@ public final class RaftClientTestContext { return this; } + Builder withKip853Rpc(boolean kip853Rpc) { + this.kip853Rpc = kip853Rpc; + return this; + } + public RaftClientTestContext build() throws IOException { Metrics metrics = new Metrics(time); MockNetworkChannel channel = new MockNetworkChannel(); @@ -257,6 +263,19 @@ public final class RaftClientTestContext { .stream() .collect(Collectors.toMap(Function.identity(), RaftClientTestContext::mockAddress)); + VoterSet voterSet = VoterSet.fromInetSocketAddresses( + channel.listenerName(), + voterAddressMap + ); + + // Compute the local listeners. Only potential voters/leader need to provide the local listeners + // If the local id is not set (must be observer), the local listener can be empty. + Endpoints localListeners = localId.isPresent() ? + voterSet.listeners(localId.getAsInt()) : + Endpoints.empty(); + + Uuid localDirectoryId = Uuid.randomUuid(); + QuorumConfig quorumConfig = new QuorumConfig( requestTimeoutMs, RETRY_BACKOFF_MS, @@ -279,6 +298,7 @@ public final class RaftClientTestContext { FETCH_MAX_WAIT_MS, clusterId.toString(), bootstrapServers, + localListeners, logContext, random, quorumConfig @@ -294,18 +314,20 @@ public final class RaftClientTestContext { RaftClientTestContext context = new RaftClientTestContext( clusterId, localId, + localDirectoryId, client, log, channel, messageQueue, time, quorumStateStore, - voters, + voterSet, IntStream .iterate(-2, id -> id - 1) .limit(bootstrapServers.size()) .boxed() .collect(Collectors.toSet()), + kip853Rpc, metrics, listener ); @@ -318,22 +340,26 @@ public final class RaftClientTestContext { } } + @SuppressWarnings("ParameterNumber") private RaftClientTestContext( Uuid clusterId, OptionalInt localId, + Uuid localDirectoryId, KafkaRaftClient client, MockLog log, MockNetworkChannel channel, MockMessageQueue messageQueue, MockTime time, QuorumStateStore quorumStateStore, - Set voters, + VoterSet voters, Set bootstrapIds, + boolean kip853Rpc, Metrics metrics, MockListener listener ) { this.clusterId = clusterId; this.localId = localId; + this.localDirectoryId = localDirectoryId; this.client = client; this.log = log; this.channel = channel; @@ -342,6 +368,7 @@ public final class RaftClientTestContext { this.quorumStateStore = quorumStateStore; this.voters = voters; this.bootstrapIds = bootstrapIds; + this.kip853Rpc = kip853Rpc; this.metrics = metrics; this.listener = listener; } @@ -444,6 +471,12 @@ public final class RaftClientTestContext { return localId.orElseThrow(() -> new AssertionError("Required local id is not defined")); } + public ReplicaKey localReplicaKey() { + return kip853Rpc ? + ReplicaKey.of(localIdOrThrow(), localDirectoryId) : + ReplicaKey.of(localIdOrThrow(), ReplicaKey.NO_DIRECTORY_ID); + } + private void expectBeginEpoch(int epoch) throws Exception { pollUntilRequest(); for (RaftRequest.Outbound request : collectBeginEpochRequests(epoch)) { @@ -472,8 +505,8 @@ public final class RaftClientTestContext { assertEquals( ElectionState.withVotedCandidate( epoch, - ReplicaKey.of(candidateId, Optional.empty()), - voters + ReplicaKey.of(candidateId, ReplicaKey.NO_DIRECTORY_ID), + voters.voterIds() ), quorumStateStore.readElectionState().get() ); @@ -481,14 +514,14 @@ public final class RaftClientTestContext { public void assertElectedLeader(int epoch, int leaderId) { assertEquals( - ElectionState.withElectedLeader(epoch, leaderId, voters), + ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), quorumStateStore.readElectionState().get() ); } void assertUnknownLeader(int epoch) { assertEquals( - ElectionState.withUnknownLeader(epoch, voters), + ElectionState.withUnknownLeader(epoch, voters.voterIds()), quorumStateStore.readElectionState().get() ); } @@ -496,7 +529,7 @@ public final class RaftClientTestContext { void assertResignedLeader(int epoch, int leaderId) { assertTrue(client.quorum().isResigned()); assertEquals( - ElectionState.withElectedLeader(epoch, leaderId, voters), + ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), quorumStateStore.readElectionState().get() ); } @@ -530,17 +563,16 @@ public final class RaftClientTestContext { .setObservers(observerStates); DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection(); - - Consumer addToNodes = replicaState -> { - if (nodes.find(replicaState.replicaId()) != null) - return; - - nodes.add(new DescribeQuorumResponseData.Node() - .setNodeId(replicaState.replicaId())); - }; - - voterStates.forEach(addToNodes); - observerStates.forEach(addToNodes); + voterStates.forEach(replicaState -> { + if (kip853Rpc && nodes.find(replicaState.replicaId()) == null) { + // KAFKA-16953 will add support for including the node listeners in the node + // collection + nodes.add( + new DescribeQuorumResponseData.Node() + .setNodeId(replicaState.replicaId()) + ); + } + }); DescribeQuorumResponseData expectedResponse = DescribeQuorumResponse.singletonResponse( metadataPartition, @@ -585,6 +617,15 @@ public final class RaftClientTestContext { assertEquals(error, Errors.forCode(partitionResponse.errorCode())); assertEquals(epoch, partitionResponse.leaderEpoch()); assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); + + if (kip853Rpc && leaderId.isPresent()) { + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId.getAsInt()); + Endpoints responseEndpoints = Endpoints.fromVoteResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } } List collectVoteRequests( @@ -610,7 +651,12 @@ public final class RaftClientTestContext { void deliverRequest(ApiMessage request) { RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound( - channel.newCorrelationId(), request.highestSupportedVersion(), request, time.milliseconds()); + channel.listenerName(), + channel.newCorrelationId(), + raftRequestVersion(request), + request, + time.milliseconds() + ); inboundRequest.completion.whenComplete((response, exception) -> { if (exception != null) { throw new RuntimeException(exception); @@ -655,6 +701,23 @@ public final class RaftClientTestContext { assertInstanceOf(BeginQuorumEpochResponseData.class, raftMessage.data()); BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) raftMessage.data(); assertEquals(responseError, Errors.forCode(response.errorCode())); + + if (!response.topics().isEmpty()) { + BeginQuorumEpochResponseData.PartitionData partitionResponse = response + .topics() + .get(0) + .partitions() + .get(0); + if (kip853Rpc && partitionResponse.leaderId() >= 0) { + int leaderId = partitionResponse.leaderId(); + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId); + Endpoints responseEndpoints = Endpoints.fromBeginQuorumEpochResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } + } } void assertSentBeginQuorumEpochResponse( @@ -675,6 +738,15 @@ public final class RaftClientTestContext { assertEquals(epoch, partitionResponse.leaderEpoch()); assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode())); + + if (kip853Rpc && leaderId.isPresent()) { + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId.getAsInt()); + Endpoints responseEndpoints = Endpoints.fromBeginQuorumEpochResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } } RaftRequest.Outbound assertSentEndQuorumEpochRequest(int epoch, int destinationId) { @@ -696,6 +768,23 @@ public final class RaftClientTestContext { assertInstanceOf(EndQuorumEpochResponseData.class, raftMessage.data()); EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) raftMessage.data(); assertEquals(responseError, Errors.forCode(response.errorCode())); + + if (!response.topics().isEmpty()) { + EndQuorumEpochResponseData.PartitionData partitionResponse = response + .topics() + .get(0) + .partitions() + .get(0); + if (kip853Rpc && partitionResponse.leaderId() >= 0) { + int leaderId = partitionResponse.leaderId(); + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId); + Endpoints responseEndpoints = Endpoints.fromEndQuorumEpochResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } + } } void assertSentEndQuorumEpochResponse( @@ -716,6 +805,15 @@ public final class RaftClientTestContext { assertEquals(epoch, partitionResponse.leaderEpoch()); assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode())); + + if (kip853Rpc && leaderId.isPresent()) { + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId.getAsInt()); + Endpoints responseEndpoints = Endpoints.fromEndQuorumEpochResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } } RaftRequest.Outbound assertSentFetchRequest() { @@ -749,7 +847,19 @@ public final class RaftClientTestContext { assertEquals(1, response.responses().size()); assertEquals(metadataPartition.topic(), response.responses().get(0).topic()); assertEquals(1, response.responses().get(0).partitions().size()); - return response.responses().get(0).partitions().get(0); + + FetchResponseData.PartitionData partitionResponse = response.responses().get(0).partitions().get(0); + if (kip853Rpc && partitionResponse.currentLeader().leaderId() >= 0) { + int leaderId = partitionResponse.currentLeader().leaderId(); + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId); + Endpoints responseEndpoints = Endpoints.fromFetchResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } + + return partitionResponse; } void assertSentFetchPartitionResponse(Errors topLevelError) { @@ -776,6 +886,7 @@ public final class RaftClientTestContext { assertEquals(-1, partitionResponse.divergingEpoch().epoch()); assertEquals(-1, partitionResponse.snapshotId().endOffset()); assertEquals(-1, partitionResponse.snapshotId().epoch()); + return (MemoryRecords) partitionResponse.records(); } @@ -802,6 +913,19 @@ public final class RaftClientTestContext { } void assertSentFetchSnapshotResponse(Errors responseError) { + assertSentFetchSnapshotResponse(responseError, metadataPartition); + } + + Optional assertSentFetchSnapshotResponse( + TopicPartition topicPartition + ) { + return assertSentFetchSnapshotResponse(Errors.NONE, topicPartition); + } + + Optional assertSentFetchSnapshotResponse( + Errors responseError, + TopicPartition topicPartition + ) { List sentMessages = drainSentResponses(ApiKeys.FETCH_SNAPSHOT); assertEquals(1, sentMessages.size()); @@ -810,19 +934,21 @@ public final class RaftClientTestContext { FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data(); assertEquals(responseError, Errors.forCode(response.errorCode())); - } - Optional assertSentFetchSnapshotResponse(TopicPartition topicPartition) { - List sentMessages = drainSentResponses(ApiKeys.FETCH_SNAPSHOT); - assertEquals(1, sentMessages.size()); + Optional result = + FetchSnapshotResponse.forTopicPartition(response, topicPartition); - RaftMessage message = sentMessages.get(0); - assertInstanceOf(FetchSnapshotResponseData.class, message.data()); + if (result.isPresent() && kip853Rpc && result.get().currentLeader().leaderId() >= 0) { + int leaderId = result.get().currentLeader().leaderId(); + Endpoints expectedLeaderEndpoints = voters.listeners(leaderId); + Endpoints responseEndpoints = Endpoints.fromFetchSnapshotResponse( + channel.listenerName(), + response.nodeEndpoints() + ); + assertEquals(expectedLeaderEndpoints, responseEndpoints); + } - FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data(); - assertEquals(Errors.NONE, Errors.forCode(response.errorCode())); - - return FetchSnapshotResponse.forTopicPartition(response, topicPartition); + return result; } List collectEndQuorumRequests( @@ -861,7 +987,7 @@ public final class RaftClientTestContext { RaftRequest.Outbound fetchRequest = assertSentFetchRequest(); int destinationId = fetchRequest.destination().id(); assertTrue( - voters.contains(destinationId) || bootstrapIds.contains(destinationId), + voters.voterIds().contains(destinationId) || bootstrapIds.contains(destinationId), String.format("id %d is not in sets %s or %s", destinationId, voters, bootstrapIds) ); assertFetchRequestData(fetchRequest, 0, 0L, 0); @@ -892,32 +1018,35 @@ public final class RaftClientTestContext { } public static InetSocketAddress mockAddress(int id) { - return new InetSocketAddress("localhost", 9990 + id); + return InetSocketAddress.createUnresolved("localhost", 9990 + id); } EndQuorumEpochResponseData endEpochResponse( int epoch, OptionalInt leaderId ) { - return EndQuorumEpochResponse.singletonResponse( + return RaftUtil.singletonEndQuorumEpochResponse( + channel.listenerName(), + endQuorumEpochRpcVersion(), Errors.NONE, metadataPartition, Errors.NONE, epoch, - leaderId.orElse(-1) + leaderId.orElse(-1), + Endpoints.empty() // KAFKA-16529 will fix this ); } EndQuorumEpochRequestData endEpochRequest( int epoch, int leaderId, - List preferredSuccessors + List preferredCandidates ) { - return EndQuorumEpochRequest.singletonRequest( - metadataPartition, + return endEpochRequest( + clusterId.toString(), epoch, leaderId, - preferredSuccessors + preferredCandidates ); } @@ -925,50 +1054,68 @@ public final class RaftClientTestContext { String clusterId, int epoch, int leaderId, - List preferredSuccessors + List preferredCandidates ) { - return EndQuorumEpochRequest.singletonRequest( + return RaftUtil.singletonEndQuorumEpochRequest( metadataPartition, clusterId, epoch, leaderId, - preferredSuccessors - ); - } - - BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) { - return BeginQuorumEpochRequest.singletonRequest( - metadataPartition, - clusterId, - epoch, - leaderId + preferredCandidates ); } BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId) { - return BeginQuorumEpochRequest.singletonRequest( + return beginEpochRequest(clusterId.toString(), epoch, leaderId); + } + + BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) { + ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID) : + ReplicaKey.of(localIdOrThrow(), localDirectoryId); + + return beginEpochRequest(clusterId, epoch, leaderId, localReplicaKey); + } + + BeginQuorumEpochRequestData beginEpochRequest( + String clusterId, + int epoch, + int leaderId, + ReplicaKey voterKey + ) { + return RaftUtil.singletonBeginQuorumEpochRequest( metadataPartition, + clusterId, epoch, - leaderId + leaderId, + voters.listeners(leaderId), + voterKey ); } private BeginQuorumEpochResponseData beginEpochResponse(int epoch, int leaderId) { - return BeginQuorumEpochResponse.singletonResponse( + return RaftUtil.singletonBeginQuorumEpochResponse( + channel.listenerName(), + beginQuorumEpochRpcVersion(), Errors.NONE, metadataPartition, Errors.NONE, epoch, - leaderId + leaderId, + Endpoints.empty() // KAFKA-16529 will fix this ); } - VoteRequestData voteRequest(int epoch, int candidateId, int lastEpoch, long lastEpochOffset) { - return VoteRequest.singletonRequest( - metadataPartition, + VoteRequestData voteRequest( + int epoch, + ReplicaKey candidateKey, + int lastEpoch, + long lastEpochOffset + ) { + return voteRequest( clusterId.toString(), epoch, - candidateId, + candidateKey, lastEpoch, lastEpochOffset ); @@ -977,28 +1124,54 @@ public final class RaftClientTestContext { VoteRequestData voteRequest( String clusterId, int epoch, - int candidateId, + ReplicaKey candidateKey, int lastEpoch, long lastEpochOffset ) { - return VoteRequest.singletonRequest( + ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID) : + ReplicaKey.of(localIdOrThrow(), localDirectoryId); + + return voteRequest( + clusterId, + epoch, + candidateKey, + localReplicaKey, + lastEpoch, + lastEpochOffset + ); + } + + VoteRequestData voteRequest( + String clusterId, + int epoch, + ReplicaKey candidateKey, + ReplicaKey voterKey, + int lastEpoch, + long lastEpochOffset + ) { + return RaftUtil.singletonVoteRequest( metadataPartition, clusterId, epoch, - candidateId, + candidateKey, + voterKey, lastEpoch, lastEpochOffset ); } VoteResponseData voteResponse(boolean voteGranted, Optional leaderId, int epoch) { - return VoteResponse.singletonResponse( + return RaftUtil.singletonVoteResponse( + channel.listenerName(), + voteRpcVersion(), Errors.NONE, metadataPartition, Errors.NONE, epoch, leaderId.orElse(-1), - voteGranted + voteGranted, + Endpoints.empty() // KAFKA-16529 will fix this ); } @@ -1062,7 +1235,7 @@ public final class RaftClientTestContext { assertEquals(localId.orElse(-1), request.replicaState().replicaId()); // Assert that voters have flushed up to the fetch offset - if (localId.isPresent() && voters.contains(localId.getAsInt())) { + if (localId.isPresent() && voters.voterIds().contains(localId.getAsInt())) { assertEquals( log.firstUnflushedOffset(), fetchOffset, @@ -1079,7 +1252,7 @@ public final class RaftClientTestContext { FetchRequestData fetchRequest( int epoch, - int replicaId, + ReplicaKey replicaKey, long fetchOffset, int lastFetchedEpoch, int maxWaitTimeMs @@ -1087,7 +1260,7 @@ public final class RaftClientTestContext { return fetchRequest( epoch, clusterId.toString(), - replicaId, + replicaKey, fetchOffset, lastFetchedEpoch, maxWaitTimeMs @@ -1097,21 +1270,28 @@ public final class RaftClientTestContext { FetchRequestData fetchRequest( int epoch, String clusterId, - int replicaId, + ReplicaKey replicaKey, long fetchOffset, int lastFetchedEpoch, int maxWaitTimeMs ) { - FetchRequestData request = RaftUtil.singletonFetchRequest(metadataPartition, metadataTopicId, fetchPartition -> { - fetchPartition + FetchRequestData request = RaftUtil.singletonFetchRequest( + metadataPartition, + metadataTopicId, + fetchPartition -> fetchPartition .setCurrentLeaderEpoch(epoch) .setLastFetchedEpoch(lastFetchedEpoch) - .setFetchOffset(fetchOffset); - }); + .setFetchOffset(fetchOffset) + .setReplicaDirectoryId( + replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID) + ) + ); return request .setMaxWaitMs(maxWaitTimeMs) .setClusterId(clusterId) - .setReplicaState(new FetchRequestData.ReplicaState().setReplicaId(replicaId)); + .setReplicaState( + new FetchRequestData.ReplicaState().setReplicaId(replicaKey.id()) + ); } FetchResponseData fetchResponse( @@ -1121,16 +1301,25 @@ public final class RaftClientTestContext { long highWatermark, Errors error ) { - return RaftUtil.singletonFetchResponse(metadataPartition, metadataTopicId, Errors.NONE, partitionData -> { - partitionData - .setRecords(records) - .setErrorCode(error.code()) - .setHighWatermark(highWatermark); + return RaftUtil.singletonFetchResponse( + channel.listenerName(), + fetchRpcVersion(), + metadataPartition, + metadataTopicId, + Errors.NONE, + leaderId, + Endpoints.empty(), // KAFKA-16529 will fix this + partitionData -> { + partitionData + .setRecords(records) + .setErrorCode(error.code()) + .setHighWatermark(highWatermark); - partitionData.currentLeader() - .setLeaderEpoch(epoch) - .setLeaderId(leaderId); - }); + partitionData.currentLeader() + .setLeaderEpoch(epoch) + .setLeaderId(leaderId); + } + ); } FetchResponseData divergingFetchResponse( @@ -1140,29 +1329,156 @@ public final class RaftClientTestContext { int divergingEpoch, long highWatermark ) { - return RaftUtil.singletonFetchResponse(metadataPartition, metadataTopicId, Errors.NONE, partitionData -> { - partitionData.setHighWatermark(highWatermark); + return RaftUtil.singletonFetchResponse( + channel.listenerName(), + fetchRpcVersion(), + metadataPartition, + metadataTopicId, + Errors.NONE, + leaderId, + Endpoints.empty(), // KAFKA-16529 will fix this + partitionData -> { + partitionData.setHighWatermark(highWatermark); - partitionData.currentLeader() - .setLeaderEpoch(epoch) - .setLeaderId(leaderId); + partitionData.currentLeader() + .setLeaderEpoch(epoch) + .setLeaderId(leaderId); - partitionData.divergingEpoch() - .setEpoch(divergingEpoch) - .setEndOffset(divergingEpochEndOffset); - }); + partitionData.divergingEpoch() + .setEpoch(divergingEpoch) + .setEndOffset(divergingEpochEndOffset); + } + ); + } + + FetchResponseData snapshotFetchResponse( + int epoch, + int leaderId, + OffsetAndEpoch snapshotId, + long highWatermark + ) { + return RaftUtil.singletonFetchResponse( + channel.listenerName(), + fetchRpcVersion(), + metadataPartition, + metadataTopicId, + Errors.NONE, + leaderId, + Endpoints.empty(), // KAFKA-16529 will fix this + partitionData -> { + partitionData.setHighWatermark(highWatermark); + + partitionData.currentLeader() + .setLeaderEpoch(epoch) + .setLeaderId(leaderId); + + partitionData.snapshotId() + .setEpoch(snapshotId.epoch()) + .setEndOffset(snapshotId.offset()); + } + ); + } + + FetchSnapshotResponseData fetchSnapshotResponse( + int leaderId, + UnaryOperator operator + ) { + return RaftUtil.singletonFetchSnapshotResponse( + channel.listenerName(), + fetchSnapshotRpcVersion(), + metadataPartition, + leaderId, + Endpoints.empty(), // KAFKA-16529 will fix this + operator + ); + } + + DescribeQuorumRequestData describeQuorumRequest() { + return RaftUtil.singletonDescribeQuorumRequest(metadataPartition); + } + + private short fetchRpcVersion() { + if (kip853Rpc) { + return 17; + } else { + return 16; + } + } + + private short fetchSnapshotRpcVersion() { + if (kip853Rpc) { + return 1; + } else { + return 0; + } + } + + private short voteRpcVersion() { + if (kip853Rpc) { + return 1; + } else { + return 0; + } + } + + private short beginQuorumEpochRpcVersion() { + if (kip853Rpc) { + return 1; + } else { + return 0; + } + } + + private short endQuorumEpochRpcVersion() { + if (kip853Rpc) { + return 1; + } else { + return 0; + } + } + + private short describeQuorumRpcVersion() { + if (kip853Rpc) { + return 2; + } else { + return 1; + } + } + + private short raftRequestVersion(ApiMessage request) { + if (request instanceof FetchRequestData) { + return fetchRpcVersion(); + } else if (request instanceof FetchSnapshotRequestData) { + return fetchSnapshotRpcVersion(); + } else if (request instanceof VoteRequestData) { + return voteRpcVersion(); + } else if (request instanceof BeginQuorumEpochRequestData) { + return beginQuorumEpochRpcVersion(); + } else if (request instanceof EndQuorumEpochRequestData) { + return endQuorumEpochRpcVersion(); + } else if (request instanceof DescribeQuorumRequestData) { + return describeQuorumRpcVersion(); + } else { + throw new IllegalArgumentException(String.format("Request %s is not a raft request", request)); + } } public void advanceLocalLeaderHighWatermarkToLogEndOffset() throws InterruptedException { assertEquals(localId, currentLeader()); long localLogEndOffset = log.endOffset().offset; - Set followers = voters.stream().filter(voter -> voter != localId.getAsInt()).collect(Collectors.toSet()); - // Send a request from every follower - for (int follower : followers) { + Iterable followers = () -> voters + .voterKeys() + .stream() + .filter(voterKey -> voterKey.id() != localId.getAsInt()) + .iterator(); + + // Send a request from every voter + for (ReplicaKey follower : followers) { deliverRequest( fetchRequest(currentEpoch(), follower, localLogEndOffset, currentEpoch(), 0) ); + pollUntilResponse(); assertSentFetchPartitionResponse(Errors.NONE, currentEpoch(), localId); } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 88c3d00c866..1b25771d31d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.message.ApiMessageType; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.Readable; @@ -772,6 +773,7 @@ public class RaftEventSimulationTest { FETCH_MAX_WAIT_MS, clusterId.toString(), Collections.emptyList(), + Endpoints.empty(), // KAFKA-16529 will fix this logContext, random, quorumConfig @@ -1236,8 +1238,20 @@ public class RaftEventSimulationTest { int correlationId = outbound.correlationId(); Node destination = outbound.destination(); - RaftRequest.Inbound inbound = new RaftRequest.Inbound(correlationId, outbound.apiVersion(), outbound.data(), - cluster.time.milliseconds()); + RaftRequest.Inbound inbound = cluster + .nodeIfRunning(senderId) + .map(node -> + new RaftRequest.Inbound( + node.channel.listenerName(), + correlationId, + ApiMessageType + .fromApiKey(outbound.data().apiKey()) + .highestSupportedVersion(true), + outbound.data(), + cluster.time.milliseconds() + ) + ) + .get(); if (!filters.get(destination.id()).acceptInbound(inbound)) return; diff --git a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java index 17a85450f99..ddc5ef1f35b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java @@ -20,18 +20,20 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.ReplicaKey; +import org.apache.kafka.raft.internals.VoterSetTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -42,6 +44,12 @@ class ResignedStateTest { int electionTimeoutMs = 5000; int localId = 0; int epoch = 5; + Endpoints localEndpoints = Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 1234) + ) + ); private ResignedState newResignedState( Set voters, @@ -54,6 +62,7 @@ class ResignedStateTest { voters, electionTimeoutMs, preferredSuccessors, + localEndpoints, logContext ); } @@ -90,9 +99,9 @@ class ResignedStateTest { Collections.emptyList() ); - assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); } @Test @@ -107,4 +116,15 @@ class ResignedStateTest { // try non-existed voter must throw an exception assertThrows(IllegalArgumentException.class, () -> state.acknowledgeResignation(10)); } + + @Test + void testLeaderEndpoints() { + ResignedState state = newResignedState( + Utils.mkSet(1, 2, 3), + Collections.emptyList() + ); + + assertEquals(localEndpoints, state.leaderEndpoints()); + assertNotEquals(Endpoints.empty(), state.leaderEndpoints()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index 3240d60107e..32d122e5175 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -78,21 +78,31 @@ public class UnattachedStateTest { @ValueSource(booleans = {true, false}) public void testGrantVote(boolean isLogUpToDate) { UnattachedState state = newUnattachedState( - Utils.mkSet(1, 2, 3), - Optional.empty() + Utils.mkSet(1, 2, 3), + Optional.empty() ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) ); } + + @Test + void testLeaderEndpoints() { + UnattachedState state = newUnattachedState( + Utils.mkSet(1, 2, 3), + Optional.empty() + ); + + assertEquals(Endpoints.empty(), state.leaderEndpoints()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java index 07eed8968bc..b8a184917a1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java @@ -41,15 +41,14 @@ class VotedStateTest { private final int electionTimeoutMs = 10000; private VotedState newVotedState( - Optional votedDirectoryId, - Optional highWatermark + Uuid votedDirectoryId ) { return new VotedState( time, epoch, ReplicaKey.of(votedId, votedDirectoryId), Collections.emptySet(), - highWatermark, + Optional.empty(), electionTimeoutMs, logContext ); @@ -57,8 +56,8 @@ class VotedStateTest { @Test public void testElectionTimeout() { - VotedState state = newVotedState(Optional.empty(), Optional.empty()); - ReplicaKey votedKey = ReplicaKey.of(votedId, Optional.empty()); + VotedState state = newVotedState(ReplicaKey.NO_DIRECTORY_ID); + ReplicaKey votedKey = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID); assertEquals(epoch, state.epoch()); assertEquals(votedKey, state.votedKey()); @@ -81,36 +80,44 @@ class VotedStateTest { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { - VotedState state = newVotedState(Optional.empty(), Optional.empty()); + VotedState state = newVotedState(ReplicaKey.NO_DIRECTORY_ID); assertTrue( - state.canGrantVote(ReplicaKey.of(votedId, Optional.empty()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) ); assertTrue( state.canGrantVote( - ReplicaKey.of(votedId, Optional.of(Uuid.randomUuid())), + ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate ) ); assertFalse( - state.canGrantVote(ReplicaKey.of(votedId + 1, Optional.empty()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) ); } @Test void testCanGrantVoteWithDirectoryId() { - Optional votedDirectoryId = Optional.of(Uuid.randomUuid()); - VotedState state = newVotedState(votedDirectoryId, Optional.empty()); + Uuid votedDirectoryId = Uuid.randomUuid(); + VotedState state = newVotedState(votedDirectoryId); assertTrue(state.canGrantVote(ReplicaKey.of(votedId, votedDirectoryId), false)); assertFalse( - state.canGrantVote(ReplicaKey.of(votedId, Optional.of(Uuid.randomUuid())), false) + state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), false) ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Optional.empty()), false)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), false)); assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), false)); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, Optional.empty()), false)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), false)); + } + + @Test + void testLeaderEndpoints() { + Uuid votedDirectoryId = Uuid.randomUuid(); + VotedState state = newVotedState(votedDirectoryId); + + assertEquals(Endpoints.empty(), state.leaderEndpoints()); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 49e749ebe5d..1b45f8c0a6f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -34,7 +34,6 @@ import org.mockito.Mockito; import java.util.Collections; import java.util.Map; -import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Random; @@ -68,9 +67,9 @@ public class KafkaRaftMetricsTest { return new QuorumState( OptionalInt.of(localId), localDirectoryId, - VoterSetTest.DEFAULT_LISTENER_NAME, () -> voterSet, () -> kraftVersion, + voterSet.listeners(localId), electionTimeoutMs, fetchTimeoutMs, new MockQuorumStateStore(), @@ -88,7 +87,7 @@ public class KafkaRaftMetricsTest { VoterSetTest.voterNode( ReplicaKey.of( localId, - withDirectoryId ? Optional.of(localDirectoryId) : Optional.empty() + withDirectoryId ? localDirectoryId : ReplicaKey.NO_DIRECTORY_ID ) ) ) @@ -105,7 +104,7 @@ public class KafkaRaftMetricsTest { VoterSetTest.voterNode( ReplicaKey.of( localId, - withDirectoryId ? Optional.of(localDirectoryId) : Optional.empty() + withDirectoryId ? localDirectoryId : ReplicaKey.NO_DIRECTORY_ID ) ) ); @@ -148,11 +147,15 @@ public class KafkaRaftMetricsTest { assertEquals((double) 1, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); - state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters.voters()); - state.leaderStateOrThrow().updateReplicaState(1, Uuid.randomUuid(), 0, new LogOffsetMetadata(5L)); + state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters); + state.leaderStateOrThrow().updateReplicaState( + voterMap.get(1).voterKey(), + 0, + new LogOffsetMetadata(5L) + ); assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToFollower(2, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); + state.transitionToFollower(2, 1, voters.listeners(1)); assertEquals("follower", getMetric(metrics, "current-state").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); @@ -166,7 +169,7 @@ public class KafkaRaftMetricsTest { state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L)); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToVoted(3, ReplicaKey.of(2, Optional.empty())); + state.transitionToVoted(3, ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)); assertEquals("voted", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) 2, getMetric(metrics, "current-vote").metricValue()); @@ -210,7 +213,7 @@ public class KafkaRaftMetricsTest { assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToFollower(2, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); + state.transitionToFollower(2, 1, voters.listeners(1)); assertEquals("observer", getMetric(metrics, "current-state").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java index 17ef319625a..ded0ac0137a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -21,8 +21,11 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.raft.Endpoints; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.net.InetSocketAddress; import java.util.Arrays; @@ -38,6 +41,7 @@ import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -101,8 +105,8 @@ public final class VoterSetTest { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); - assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Optional.empty()))); - assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Optional.of(Uuid.randomUuid())))); + assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID))); + assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Uuid.randomUuid()))); VoterSet.VoterNode voter3 = aVoterMap.remove(3); assertEquals( @@ -117,15 +121,15 @@ public final class VoterSetTest { VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey())); - assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); - assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.empty()))); + assertFalse(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertFalse(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); assertFalse( - voterSet.isVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId())) + voterSet.isVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId().get())) ); assertFalse( - voterSet.isVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId())) + voterSet.isVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId().get())) ); - assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty()))); + assertFalse(voterSet.isVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID))); } @Test @@ -133,10 +137,46 @@ public final class VoterSetTest { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), false); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); - assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.empty()))); - assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); - assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.of(Uuid.randomUuid())))); - assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty()))); + assertTrue(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); + assertTrue(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertFalse(voterSet.isVoter(ReplicaKey.of(4, Uuid.randomUuid()))); + assertFalse(voterSet.isVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID))); + } + + @Test + void testVoterNodeIsVoterWithDirectoryId() { + VoterSet.VoterNode voterNode = voterNode(1, true); + + assertTrue(voterNode.isVoter(voterNode.voterKey())); + assertFalse(voterNode.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertFalse(voterNode.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); + assertFalse(voterNode.isVoter(ReplicaKey.of(2, Uuid.randomUuid()))); + assertFalse(voterNode.isVoter(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID))); + assertFalse(voterNode.isVoter(ReplicaKey.of(2, voterNode.voterKey().directoryId().get()))); + } + + @Test + void testVoterNodeIsVoterWithoutDirectoryId() { + VoterSet.VoterNode voterNode = voterNode(1, false); + + assertTrue(voterNode.isVoter(voterNode.voterKey())); + assertTrue(voterNode.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertTrue(voterNode.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); + assertTrue(voterNode.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertFalse(voterNode.isVoter(ReplicaKey.of(2, Uuid.randomUuid()))); + assertFalse(voterNode.isVoter(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID))); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + void testEndpoints(boolean withDirectoryId) { + Map aVoterMap = voterMap(IntStream.of(1, 2, 3), withDirectoryId); + VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + + assertNotEquals(Endpoints.empty(), voterSet.listeners(1)); + assertNotEquals(Endpoints.empty(), voterSet.listeners(2)); + assertNotEquals(Endpoints.empty(), voterSet.listeners(3)); + assertEquals(Endpoints.empty(), voterSet.listeners(4)); } @Test @@ -145,12 +185,12 @@ public final class VoterSetTest { VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); - assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); - assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty()))); + assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); assertFalse( - voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId())) + voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId().get())) ); - assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, Optional.empty()))); + assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID))); } @Test @@ -159,15 +199,15 @@ public final class VoterSetTest { VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertFalse(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); - assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); - assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty()))); + assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid()))); + assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); assertFalse( - voterSet.isOnlyVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId())) + voterSet.isOnlyVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId().get())) ); assertFalse( - voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId())) + voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId().get())) ); - assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, Optional.empty()))); + assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID))); } @Test @@ -272,7 +312,7 @@ public final class VoterSetTest { return voterNode( ReplicaKey.of( id, - withDirectoryId ? Optional.of(Uuid.randomUuid()) : Optional.empty() + withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID ) ); } @@ -280,11 +320,13 @@ public final class VoterSetTest { public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { return new VoterSet.VoterNode( replicaKey, - Collections.singletonMap( - DEFAULT_LISTENER_NAME, - InetSocketAddress.createUnresolved( - String.format("replica-%d", replicaKey.id()), - 1234 + Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved( + String.format("replica-%d", replicaKey.id()), + 1234 + ) ) ), new SupportedVersionRange((short) 0, (short) 0) diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 90ac27feeea..e7549dd8bad 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -408,7 +408,10 @@ public enum MetadataVersion { } public short fetchRequestVersion() { - if (this.isAtLeast(IBP_3_7_IV4)) { + if (this.isAtLeast(IBP_4_0_IV0)) { + // KAFKA-17018 will change this to IBP_3_9_IVX when it is available + return 17; + } else if (this.isAtLeast(IBP_3_7_IV4)) { return 16; } else if (this.isAtLeast(IBP_3_5_IV1)) { return 15;