KAFKA-8471: Replace control requests/responses with automated protocol (#7353)

Replaced UpdateMetadata{Request, Response}, LeaderAndIsr{Request, Response}
and StopReplica{Request, Response} with the automated protocol classes.

Updated the JSON schema for the 3 request types to be more consistent and
less strict (if needed to avoid duplication).

The general approach is to avoid generating new collections in the request
classes. Normalization happens in the constructor to make this possible. Builders
still have to group by topic to maintain the external ungrouped view.

Introduced new tests for LeaderAndIsrRequest and UpdateMetadataRequest to
verify that the new logic is correct.

A few other clean-ups/fixes in code that was touched due to these changes:
* KAFKA-8956: Refactor DelayedCreatePartitions#updateWaiting to avoid modifying
collection in foreach.
* Avoid unnecessary allocation for state change trace logging if trace logging is not enabled
* Use `toBuffer` instead of `toList`, `toIndexedSeq` or `toSeq` as it generally performs
better and it matches the performance characteristics of `java.util.ArrayList`. This is
particularly important when passing such instances to Java code.
* Minor refactoring for clarity and readability.
* Removed usage of deprecated `/:`, unused imports and unnecessary `var`s.
* Include exception in `AdminClientIntegrationTest` failure message.
* Move StopReplicaRequest verification in `AuthorizerIntegrationTest` to the end
to match the comment.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
This commit is contained in:
Ismael Juma 2019-09-28 19:39:45 -07:00 committed by GitHub
parent 1dc5063cde
commit 66183f730f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
65 changed files with 2366 additions and 1780 deletions

View File

@ -44,6 +44,8 @@ import org.apache.kafka.common.message.InitProducerIdRequestData;
import org.apache.kafka.common.message.InitProducerIdResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.message.JoinGroupResponseData;
import org.apache.kafka.common.message.LeaderAndIsrRequestData;
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaveGroupRequestData;
import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.message.ListGroupsRequestData;
@ -66,8 +68,12 @@ import org.apache.kafka.common.message.SaslAuthenticateRequestData;
import org.apache.kafka.common.message.SaslAuthenticateResponseData;
import org.apache.kafka.common.message.SaslHandshakeRequestData;
import org.apache.kafka.common.message.SaslHandshakeResponseData;
import org.apache.kafka.common.message.StopReplicaRequestData;
import org.apache.kafka.common.message.StopReplicaResponseData;
import org.apache.kafka.common.message.SyncGroupRequestData;
import org.apache.kafka.common.message.SyncGroupResponseData;
import org.apache.kafka.common.message.UpdateMetadataRequestData;
import org.apache.kafka.common.message.UpdateMetadataResponseData;
import org.apache.kafka.common.message.TxnOffsetCommitRequestData;
import org.apache.kafka.common.message.TxnOffsetCommitResponseData;
import org.apache.kafka.common.protocol.types.Schema;
@ -103,18 +109,12 @@ import org.apache.kafka.common.requests.EndTxnRequest;
import org.apache.kafka.common.requests.EndTxnResponse;
import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
import org.apache.kafka.common.requests.LeaderAndIsrResponse;
import org.apache.kafka.common.requests.ListOffsetRequest;
import org.apache.kafka.common.requests.ListOffsetResponse;
import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse;
import org.apache.kafka.common.requests.ProduceRequest;
import org.apache.kafka.common.requests.ProduceResponse;
import org.apache.kafka.common.requests.StopReplicaRequest;
import org.apache.kafka.common.requests.StopReplicaResponse;
import org.apache.kafka.common.requests.UpdateMetadataRequest;
import org.apache.kafka.common.requests.UpdateMetadataResponse;
import org.apache.kafka.common.requests.WriteTxnMarkersRequest;
import org.apache.kafka.common.requests.WriteTxnMarkersResponse;
@ -133,10 +133,9 @@ public enum ApiKeys {
FETCH(1, "Fetch", FetchRequest.schemaVersions(), FetchResponse.schemaVersions()),
LIST_OFFSETS(2, "ListOffsets", ListOffsetRequest.schemaVersions(), ListOffsetResponse.schemaVersions()),
METADATA(3, "Metadata", MetadataRequestData.SCHEMAS, MetadataResponseData.SCHEMAS),
LEADER_AND_ISR(4, "LeaderAndIsr", true, LeaderAndIsrRequest.schemaVersions(), LeaderAndIsrResponse.schemaVersions()),
STOP_REPLICA(5, "StopReplica", true, StopReplicaRequest.schemaVersions(), StopReplicaResponse.schemaVersions()),
UPDATE_METADATA(6, "UpdateMetadata", true, UpdateMetadataRequest.schemaVersions(),
UpdateMetadataResponse.schemaVersions()),
LEADER_AND_ISR(4, "LeaderAndIsr", true, LeaderAndIsrRequestData.SCHEMAS, LeaderAndIsrResponseData.SCHEMAS),
STOP_REPLICA(5, "StopReplica", true, StopReplicaRequestData.SCHEMAS, StopReplicaResponseData.SCHEMAS),
UPDATE_METADATA(6, "UpdateMetadata", true, UpdateMetadataRequestData.SCHEMAS, UpdateMetadataResponseData.SCHEMAS),
CONTROLLED_SHUTDOWN(7, "ControlledShutdown", true, ControlledShutdownRequestData.SCHEMAS,
ControlledShutdownResponseData.SCHEMAS),
OFFSET_COMMIT(8, "OffsetCommit", OffsetCommitRequestData.SCHEMAS, OffsetCommitResponseData.SCHEMAS),

View File

@ -471,6 +471,7 @@ public class Struct {
* @throws SchemaException If validation fails
*/
private void validateField(BoundField field) {
Objects.requireNonNull(field, "`field` must be non-null");
if (this.schema != field.schema)
throw new SchemaException("Attempt to access field '" + field.def.name + "' from a different schema instance.");
if (field.index > values.length)

View File

@ -17,21 +17,12 @@
package org.apache.kafka.common.requests;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Struct;
// Abstract class for all control requests including UpdateMetadataRequest, LeaderAndIsrRequest and StopReplicaRequest
public abstract class AbstractControlRequest extends AbstractRequest {
public static final long UNKNOWN_BROKER_EPOCH = -1L;
protected static final Field.Int32 CONTROLLER_ID = new Field.Int32("controller_id", "The controller id");
protected static final Field.Int32 CONTROLLER_EPOCH = new Field.Int32("controller_epoch", "The controller epoch");
protected static final Field.Int64 BROKER_EPOCH = new Field.Int64("broker_epoch", "The broker epoch");
protected final int controllerId;
protected final int controllerEpoch;
protected final long brokerEpoch;
public static abstract class Builder<T extends AbstractRequest> extends AbstractRequest.Builder<T> {
protected final int controllerId;
protected final int controllerEpoch;
@ -46,35 +37,14 @@ public abstract class AbstractControlRequest extends AbstractRequest {
}
public int controllerId() {
return controllerId;
}
public int controllerEpoch() {
return controllerEpoch;
}
public long brokerEpoch() {
return brokerEpoch;
}
protected AbstractControlRequest(ApiKeys api, short version, int controllerId, int controllerEpoch, long brokerEpoch) {
protected AbstractControlRequest(ApiKeys api, short version) {
super(api, version);
this.controllerId = controllerId;
this.controllerEpoch = controllerEpoch;
this.brokerEpoch = brokerEpoch;
}
protected AbstractControlRequest(ApiKeys api, Struct struct, short version) {
super(api, version);
this.controllerId = struct.get(CONTROLLER_ID);
this.controllerEpoch = struct.get(CONTROLLER_EPOCH);
this.brokerEpoch = struct.getOrElse(BROKER_EPOCH, UNKNOWN_BROKER_EPOCH);
}
public abstract int controllerId();
// Used for test
long size() {
return toStruct().sizeOf();
}
public abstract int controllerEpoch();
public abstract long brokerEpoch();
}

View File

@ -23,6 +23,7 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@ -55,9 +56,9 @@ public abstract class AbstractResponse extends AbstractRequestResponse {
return Collections.singletonMap(error, 1);
}
protected Map<Errors, Integer> errorCounts(Map<?, Errors> errors) {
protected Map<Errors, Integer> errorCounts(Collection<Errors> errors) {
Map<Errors, Integer> errorCounts = new HashMap<>();
for (Errors error : errors.values())
for (Errors error : errors)
updateErrorCounts(errorCounts, error);
return errorCounts;
}
@ -101,13 +102,13 @@ public abstract class AbstractResponse extends AbstractRequestResponse {
case SYNC_GROUP:
return new SyncGroupResponse(struct, version);
case STOP_REPLICA:
return new StopReplicaResponse(struct);
return new StopReplicaResponse(struct, version);
case CONTROLLED_SHUTDOWN:
return new ControlledShutdownResponse(struct, version);
case UPDATE_METADATA:
return new UpdateMetadataResponse(struct);
return new UpdateMetadataResponse(struct, version);
case LEADER_AND_ISR:
return new LeaderAndIsrResponse(struct);
return new LeaderAndIsrResponse(struct, version);
case DESCRIBE_GROUPS:
return new DescribeGroupsResponse(struct, version);
case LIST_GROUPS:

View File

@ -102,7 +102,7 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
@Override
public Map<Errors, Integer> errorCounts() {
return errorCounts(errors);
return errorCounts(errors.values());
}
@Override

View File

@ -132,7 +132,7 @@ public class AlterReplicaLogDirsResponse extends AbstractResponse {
@Override
public Map<Errors, Integer> errorCounts() {
return errorCounts(responses);
return errorCounts(responses.values());
}
public static AlterReplicaLogDirsResponse parse(ByteBuffer buffer, short version) {

View File

@ -17,152 +17,37 @@
package org.apache.kafka.common.requests;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.LeaderAndIsrRequestData;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrLiveLeader;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrTopicState;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.FlattenedIterator;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Collections;
import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
import static org.apache.kafka.common.protocol.types.Type.INT32;
import java.util.stream.Collectors;
public class LeaderAndIsrRequest extends AbstractControlRequest {
private static final Field.ComplexArray TOPIC_STATES = new Field.ComplexArray("topic_states", "Topic states");
private static final Field.ComplexArray PARTITION_STATES = new Field.ComplexArray("partition_states", "Partition states");
private static final Field.ComplexArray LIVE_LEADERS = new Field.ComplexArray("live_leaders", "Live leaders");
// PartitionState fields
private static final Field.Int32 LEADER = new Field.Int32("leader", "The broker id for the leader.");
private static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The leader epoch.");
private static final Field.Array ISR = new Field.Array("isr", INT32, "The in sync replica ids.");
private static final Field.Int32 ZK_VERSION = new Field.Int32("zk_version", "The ZK version.");
private static final Field.Array REPLICAS = new Field.Array("replicas", INT32, "The replica ids.");
private static final Field.Array ADDING_REPLICAS = new Field.Array("adding_replicas", INT32,
"The replica ids we are in the process of adding to the replica set during a reassignment.");
private static final Field.Array REMOVING_REPLICAS = new Field.Array("removing_replicas", INT32,
"The replica ids we are in the process of removing from the replica set during a reassignment.");
private static final Field.Bool IS_NEW = new Field.Bool("is_new", "Whether the replica should have existed on the broker or not");
// live_leaders fields
private static final Field.Int32 END_POINT_ID = new Field.Int32("id", "The broker id");
private static final Field.Str HOST = new Field.Str("host", "The hostname of the broker.");
private static final Field.Int32 PORT = new Field.Int32("port", "The port on which the broker accepts requests.");
private static final Field PARTITION_STATES_V0 = PARTITION_STATES.withFields(
TOPIC_NAME,
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS);
// PARTITION_STATES_V1 added a per-partition is_new Field.
// This field specifies whether the replica should have existed on the broker or not.
private static final Field PARTITION_STATES_V1 = PARTITION_STATES.withFields(
TOPIC_NAME,
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS,
IS_NEW);
private static final Field PARTITION_STATES_V2 = PARTITION_STATES.withFields(
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS,
IS_NEW);
private static final Field PARTITION_STATES_V3 = PARTITION_STATES.withFields(
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS,
ADDING_REPLICAS,
REMOVING_REPLICAS,
IS_NEW);
// TOPIC_STATES_V2 normalizes TOPIC_STATES_V1 to make it more memory efficient
private static final Field TOPIC_STATES_V2 = TOPIC_STATES.withFields(
TOPIC_NAME,
PARTITION_STATES_V2);
// TOPIC_STATES_V3 adds two new fields - adding_replicas and removing_replicas
private static final Field TOPIC_STATES_V3 = TOPIC_STATES.withFields(
TOPIC_NAME,
PARTITION_STATES_V3);
private static final Field LIVE_LEADERS_V0 = LIVE_LEADERS.withFields(
END_POINT_ID,
HOST,
PORT);
private static final Schema LEADER_AND_ISR_REQUEST_V0 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V0,
LIVE_LEADERS_V0);
// LEADER_AND_ISR_REQUEST_V1 added a per-partition is_new Field. This field specifies whether the replica should
// have existed on the broker or not.
private static final Schema LEADER_AND_ISR_REQUEST_V1 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V1,
LIVE_LEADERS_V0);
// LEADER_AND_ISR_REQUEST_V2 added a broker_epoch Field. This field specifies the generation of the broker across
// bounces. It also normalizes partitions under each topic.
private static final Schema LEADER_AND_ISR_REQUEST_V2 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
BROKER_EPOCH,
TOPIC_STATES_V2,
LIVE_LEADERS_V0);
// LEADER_AND_ISR_REQUEST_V3 added two new fields - adding_replicas and removing_replicas.
// These fields respectively specify the replica IDs we want to add or remove as part of a reassignment
private static final Schema LEADER_AND_ISR_REQUEST_V3 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
BROKER_EPOCH,
TOPIC_STATES_V3,
LIVE_LEADERS_V0);
public static Schema[] schemaVersions() {
return new Schema[]{LEADER_AND_ISR_REQUEST_V0, LEADER_AND_ISR_REQUEST_V1, LEADER_AND_ISR_REQUEST_V2, LEADER_AND_ISR_REQUEST_V3};
}
public static class Builder extends AbstractControlRequest.Builder<LeaderAndIsrRequest> {
private final Map<TopicPartition, PartitionState> partitionStates;
private final Set<Node> liveLeaders;
private final List<LeaderAndIsrPartitionState> partitionStates;
private final Collection<Node> liveLeaders;
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
Map<TopicPartition, PartitionState> partitionStates, Set<Node> liveLeaders) {
List<LeaderAndIsrPartitionState> partitionStates, Collection<Node> liveLeaders) {
super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch);
this.partitionStates = partitionStates;
this.liveLeaders = liveLeaders;
@ -170,7 +55,38 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
@Override
public LeaderAndIsrRequest build(short version) {
return new LeaderAndIsrRequest(controllerId, controllerEpoch, brokerEpoch, partitionStates, liveLeaders, version);
List<LeaderAndIsrLiveLeader> leaders = liveLeaders.stream().map(n -> new LeaderAndIsrLiveLeader()
.setBrokerId(n.id())
.setHostName(n.host())
.setPort(n.port())
).collect(Collectors.toList());
LeaderAndIsrRequestData data = new LeaderAndIsrRequestData()
.setControllerId(controllerId)
.setControllerEpoch(controllerEpoch)
.setBrokerEpoch(brokerEpoch)
.setLiveLeaders(leaders);
if (version >= 2) {
Map<String, LeaderAndIsrTopicState> topicStatesMap = groupByTopic(partitionStates);
data.setTopicStates(new ArrayList<>(topicStatesMap.values()));
} else {
data.setUngroupedPartitionStates(partitionStates);
}
return new LeaderAndIsrRequest(data, version);
}
private static Map<String, LeaderAndIsrTopicState> groupByTopic(List<LeaderAndIsrPartitionState> partitionStates) {
Map<String, LeaderAndIsrTopicState> topicStates = new HashMap<>();
// We don't null out the topic name in LeaderAndIsrRequestPartition since it's ignored by
// the generated code if version >= 2
for (LeaderAndIsrPartitionState partition : partitionStates) {
LeaderAndIsrTopicState topicState = topicStates.computeIfAbsent(partition.topicName(),
t -> new LeaderAndIsrTopicState().setTopicName(partition.topicName()));
topicState.partitionStates().add(partition);
}
return topicStates;
}
@Override
@ -184,116 +100,60 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
.append(", liveLeaders=(").append(Utils.join(liveLeaders, ", ")).append(")")
.append(")");
return bld.toString();
}
}
private final Map<TopicPartition, PartitionState> partitionStates;
private final Set<Node> liveLeaders;
private final LeaderAndIsrRequestData data;
private LeaderAndIsrRequest(int controllerId, int controllerEpoch, long brokerEpoch, Map<TopicPartition, PartitionState> partitionStates,
Set<Node> liveLeaders, short version) {
super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch);
this.partitionStates = partitionStates;
this.liveLeaders = liveLeaders;
LeaderAndIsrRequest(LeaderAndIsrRequestData data, short version) {
super(ApiKeys.LEADER_AND_ISR, version);
this.data = data;
// Do this from the constructor to make it thread-safe (even though it's only needed when some methods are called)
normalize();
}
private void normalize() {
if (version() >= 2) {
for (LeaderAndIsrTopicState topicState : data.topicStates()) {
for (LeaderAndIsrPartitionState partitionState : topicState.partitionStates()) {
// Set the topic name so that we can always present the ungrouped view to callers
partitionState.setTopicName(topicState.topicName());
}
}
}
}
public LeaderAndIsrRequest(Struct struct, short version) {
super(ApiKeys.LEADER_AND_ISR, struct, version);
Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
if (struct.hasField(TOPIC_STATES)) {
for (Object topicStatesDataObj : struct.get(TOPIC_STATES)) {
Struct topicStatesData = (Struct) topicStatesDataObj;
String topic = topicStatesData.get(TOPIC_NAME);
for (Object partitionStateDataObj : topicStatesData.get(PARTITION_STATES)) {
Struct partitionStateData = (Struct) partitionStateDataObj;
int partition = partitionStateData.get(PARTITION_ID);
PartitionState partitionState = new PartitionState(partitionStateData);
partitionStates.put(new TopicPartition(topic, partition), partitionState);
}
}
} else {
for (Object partitionStateDataObj : struct.get(PARTITION_STATES)) {
Struct partitionStateData = (Struct) partitionStateDataObj;
String topic = partitionStateData.get(TOPIC_NAME);
int partition = partitionStateData.get(PARTITION_ID);
PartitionState partitionState = new PartitionState(partitionStateData);
partitionStates.put(new TopicPartition(topic, partition), partitionState);
}
}
Set<Node> leaders = new HashSet<>();
for (Object leadersDataObj : struct.get(LIVE_LEADERS)) {
Struct leadersData = (Struct) leadersDataObj;
int id = leadersData.get(END_POINT_ID);
String host = leadersData.get(HOST);
int port = leadersData.get(PORT);
leaders.add(new Node(id, host, port));
}
this.partitionStates = partitionStates;
this.liveLeaders = leaders;
this(new LeaderAndIsrRequestData(struct, version), version);
}
@Override
protected Struct toStruct() {
short version = version();
Struct struct = new Struct(ApiKeys.LEADER_AND_ISR.requestSchema(version));
struct.set(CONTROLLER_ID, controllerId);
struct.set(CONTROLLER_EPOCH, controllerEpoch);
struct.setIfExists(BROKER_EPOCH, brokerEpoch);
return data.toStruct(version());
}
if (struct.hasField(TOPIC_STATES)) {
Map<String, Map<Integer, PartitionState>> topicStates = CollectionUtils.groupPartitionDataByTopic(partitionStates);
List<Struct> topicStatesData = new ArrayList<>(topicStates.size());
for (Map.Entry<String, Map<Integer, PartitionState>> entry : topicStates.entrySet()) {
Struct topicStateData = struct.instance(TOPIC_STATES);
topicStateData.set(TOPIC_NAME, entry.getKey());
Map<Integer, PartitionState> partitionMap = entry.getValue();
List<Struct> partitionStatesData = new ArrayList<>(partitionMap.size());
for (Map.Entry<Integer, PartitionState> partitionEntry : partitionMap.entrySet()) {
Struct partitionStateData = topicStateData.instance(PARTITION_STATES);
partitionStateData.set(PARTITION_ID, partitionEntry.getKey());
partitionEntry.getValue().setStruct(partitionStateData, version);
partitionStatesData.add(partitionStateData);
}
topicStateData.set(PARTITION_STATES, partitionStatesData.toArray());
topicStatesData.add(topicStateData);
}
struct.set(TOPIC_STATES, topicStatesData.toArray());
} else {
List<Struct> partitionStatesData = new ArrayList<>(partitionStates.size());
for (Map.Entry<TopicPartition, PartitionState> entry : partitionStates.entrySet()) {
Struct partitionStateData = struct.instance(PARTITION_STATES);
TopicPartition topicPartition = entry.getKey();
partitionStateData.set(TOPIC_NAME, topicPartition.topic());
partitionStateData.set(PARTITION_ID, topicPartition.partition());
entry.getValue().setStruct(partitionStateData, version);
partitionStatesData.add(partitionStateData);
}
struct.set(PARTITION_STATES, partitionStatesData.toArray());
}
List<Struct> leadersData = new ArrayList<>(liveLeaders.size());
for (Node leader : liveLeaders) {
Struct leaderData = struct.instance(LIVE_LEADERS);
leaderData.set(END_POINT_ID, leader.id());
leaderData.set(HOST, leader.host());
leaderData.set(PORT, leader.port());
leadersData.add(leaderData);
}
struct.set(LIVE_LEADERS, leadersData.toArray());
return struct;
protected ByteBuffer toBytes() {
ByteBuffer bytes = ByteBuffer.allocate(size());
data.write(new ByteBufferAccessor(bytes), version());
bytes.flip();
return bytes;
}
@Override
public LeaderAndIsrResponse getErrorResponse(int throttleTimeMs, Throwable e) {
LeaderAndIsrResponseData responseData = new LeaderAndIsrResponseData();
Errors error = Errors.forException(e);
responseData.setErrorCode(error.code());
Map<TopicPartition, Errors> responses = new HashMap<>(partitionStates.size());
for (TopicPartition partition : partitionStates.keySet()) {
responses.put(partition, error);
List<LeaderAndIsrPartitionError> partitions = new ArrayList<>();
for (LeaderAndIsrPartitionState partition : partitionStates()) {
partitions.add(new LeaderAndIsrPartitionError()
.setTopicName(partition.topicName())
.setPartitionIndex(partition.partitionIndex())
.setErrorCode(error.code()));
}
responseData.setPartitionErrors(partitions);
short versionId = version();
switch (versionId) {
@ -301,133 +161,45 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
case 1:
case 2:
case 3:
return new LeaderAndIsrResponse(error, responses);
return new LeaderAndIsrResponse(responseData);
default:
throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
versionId, this.getClass().getSimpleName(), ApiKeys.LEADER_AND_ISR.latestVersion()));
}
}
@Override
public int controllerId() {
return controllerId;
return data.controllerId();
}
@Override
public int controllerEpoch() {
return controllerEpoch;
return data.controllerEpoch();
}
public Map<TopicPartition, PartitionState> partitionStates() {
return partitionStates;
@Override
public long brokerEpoch() {
return data.brokerEpoch();
}
public Set<Node> liveLeaders() {
return liveLeaders;
public Iterable<LeaderAndIsrPartitionState> partitionStates() {
if (version() >= 2)
return () -> new FlattenedIterator<>(data.topicStates().iterator(),
topicState -> topicState.partitionStates().iterator());
return data.ungroupedPartitionStates();
}
public List<LeaderAndIsrLiveLeader> liveLeaders() {
return Collections.unmodifiableList(data.liveLeaders());
}
protected int size() {
return data.size(version());
}
public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) {
return new LeaderAndIsrRequest(ApiKeys.LEADER_AND_ISR.parseRequest(version, buffer), version);
}
public static final class PartitionState {
public final BasePartitionState basePartitionState;
public final List<Integer> addingReplicas;
public final List<Integer> removingReplicas;
public final boolean isNew;
public PartitionState(int controllerEpoch,
int leader,
int leaderEpoch,
List<Integer> isr,
int zkVersion,
List<Integer> replicas,
boolean isNew) {
this(controllerEpoch,
leader,
leaderEpoch,
isr,
zkVersion,
replicas,
Collections.emptyList(),
Collections.emptyList(),
isNew);
}
public PartitionState(int controllerEpoch,
int leader,
int leaderEpoch,
List<Integer> isr,
int zkVersion,
List<Integer> replicas,
List<Integer> addingReplicas,
List<Integer> removingReplicas,
boolean isNew) {
this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas);
this.addingReplicas = addingReplicas;
this.removingReplicas = removingReplicas;
this.isNew = isNew;
}
private PartitionState(Struct struct) {
int controllerEpoch = struct.get(CONTROLLER_EPOCH);
int leader = struct.get(LEADER);
int leaderEpoch = struct.get(LEADER_EPOCH);
Object[] isrArray = struct.get(ISR);
List<Integer> isr = new ArrayList<>(isrArray.length);
for (Object r : isrArray)
isr.add((Integer) r);
int zkVersion = struct.get(ZK_VERSION);
Object[] replicasArray = struct.get(REPLICAS);
List<Integer> replicas = new ArrayList<>(replicasArray.length);
for (Object r : replicasArray)
replicas.add((Integer) r);
this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas);
List<Integer> addingReplicas = new ArrayList<>();
if (struct.hasField(ADDING_REPLICAS)) {
for (Object r : struct.get(ADDING_REPLICAS))
addingReplicas.add((Integer) r);
}
this.addingReplicas = addingReplicas;
List<Integer> removingReplicas = new ArrayList<>();
if (struct.hasField(REMOVING_REPLICAS)) {
for (Object r : struct.get(REMOVING_REPLICAS))
removingReplicas.add((Integer) r);
}
this.removingReplicas = removingReplicas;
this.isNew = struct.getOrElse(IS_NEW, false);
}
@Override
public String toString() {
return "PartitionState(controllerEpoch=" + basePartitionState.controllerEpoch +
", leader=" + basePartitionState.leader +
", leaderEpoch=" + basePartitionState.leaderEpoch +
", isr=" + Utils.join(basePartitionState.isr, ",") +
", zkVersion=" + basePartitionState.zkVersion +
", replicas=" + Utils.join(basePartitionState.replicas, ",") +
", addingReplicas=" + Utils.join(addingReplicas, ",") +
", removingReplicas=" + Utils.join(removingReplicas, ",") +
", isNew=" + isNew + ")";
}
private void setStruct(Struct struct, short version) {
struct.set(CONTROLLER_EPOCH, basePartitionState.controllerEpoch);
struct.set(LEADER, basePartitionState.leader);
struct.set(LEADER_EPOCH, basePartitionState.leaderEpoch);
struct.set(ISR, basePartitionState.isr.toArray());
struct.set(ZK_VERSION, basePartitionState.zkVersion);
struct.set(REPLICAS, basePartitionState.replicas.toArray());
if (version >= 3) {
struct.set(ADDING_REPLICAS, addingReplicas.toArray());
struct.set(REMOVING_REPLICAS, removingReplicas.toArray());
}
struct.setIfExists(IS_NEW, isNew);
}
}
}

View File

@ -16,45 +16,19 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
import java.util.stream.Collectors;
public class LeaderAndIsrResponse extends AbstractResponse {
private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions", "Response for the requests partitions");
private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
TOPIC_NAME,
PARTITION_ID,
ERROR_CODE);
private static final Schema LEADER_AND_ISR_RESPONSE_V0 = new Schema(
ERROR_CODE,
PARTITIONS_V0);
// LeaderAndIsrResponse V1 may receive KAFKA_STORAGE_ERROR in the response
private static final Schema LEADER_AND_ISR_RESPONSE_V1 = LEADER_AND_ISR_RESPONSE_V0;
private static final Schema LEADER_AND_ISR_RESPONSE_V2 = LEADER_AND_ISR_RESPONSE_V1;
private static final Schema LEADER_AND_ISR_RESPONSE_V3 = LEADER_AND_ISR_RESPONSE_V2;
public static Schema[] schemaVersions() {
return new Schema[]{LEADER_AND_ISR_RESPONSE_V0, LEADER_AND_ISR_RESPONSE_V1, LEADER_AND_ISR_RESPONSE_V2, LEADER_AND_ISR_RESPONSE_V3};
}
/**
* Possible error code:
@ -62,74 +36,45 @@ public class LeaderAndIsrResponse extends AbstractResponse {
* STALE_CONTROLLER_EPOCH (11)
* STALE_BROKER_EPOCH (77)
*/
private final Errors error;
private final LeaderAndIsrResponseData data;
private final Map<TopicPartition, Errors> responses;
public LeaderAndIsrResponse(Errors error, Map<TopicPartition, Errors> responses) {
this.responses = responses;
this.error = error;
public LeaderAndIsrResponse(LeaderAndIsrResponseData data) {
this.data = data;
}
public LeaderAndIsrResponse(Struct struct) {
responses = new HashMap<>();
for (Object responseDataObj : struct.get(PARTITIONS)) {
Struct responseData = (Struct) responseDataObj;
String topic = responseData.get(TOPIC_NAME);
int partition = responseData.get(PARTITION_ID);
Errors error = Errors.forCode(responseData.get(ERROR_CODE));
responses.put(new TopicPartition(topic, partition), error);
}
error = Errors.forCode(struct.get(ERROR_CODE));
public LeaderAndIsrResponse(Struct struct, short version) {
this.data = new LeaderAndIsrResponseData(struct, version);
}
public Map<TopicPartition, Errors> responses() {
return responses;
public List<LeaderAndIsrPartitionError> partitions() {
return data.partitionErrors();
}
public Errors error() {
return error;
return Errors.forCode(data.errorCode());
}
@Override
public Map<Errors, Integer> errorCounts() {
Errors error = error();
if (error != Errors.NONE)
// Minor optimization since the top-level error applies to all partitions
return Collections.singletonMap(error, responses.size());
return errorCounts(responses);
return Collections.singletonMap(error, data.partitionErrors().size());
return errorCounts(data.partitionErrors().stream().map(l -> Errors.forCode(l.errorCode())).collect(Collectors.toList()));
}
public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) {
return new LeaderAndIsrResponse(ApiKeys.LEADER_AND_ISR.parseResponse(version, buffer));
return new LeaderAndIsrResponse(ApiKeys.LEADER_AND_ISR.parseResponse(version, buffer), version);
}
@Override
protected Struct toStruct(short version) {
Struct struct = new Struct(ApiKeys.LEADER_AND_ISR.responseSchema(version));
List<Struct> responseDatas = new ArrayList<>(responses.size());
for (Map.Entry<TopicPartition, Errors> response : responses.entrySet()) {
Struct partitionData = struct.instance(PARTITIONS);
TopicPartition partition = response.getKey();
partitionData.set(TOPIC_NAME, partition.topic());
partitionData.set(PARTITION_ID, partition.partition());
partitionData.set(ERROR_CODE, response.getValue().code());
responseDatas.add(partitionData);
}
struct.set(PARTITIONS, responseDatas.toArray());
struct.set(ERROR_CODE, error.code());
return struct;
return data.toStruct(version);
}
@Override
public String toString() {
return "LeaderAndIsrResponse(" +
"responses=" + responses +
", error=" + error +
")";
return data.toString();
}
}

View File

@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
@ -94,14 +95,13 @@ public class OffsetCommitResponse extends AbstractResponse {
@Override
public Map<Errors, Integer> errorCounts() {
Map<TopicPartition, Errors> errorMap = new HashMap<>();
List<Errors> errors = new ArrayList<>();
for (OffsetCommitResponseTopic topic : data.topics()) {
for (OffsetCommitResponsePartition partition : topic.partitions()) {
errorMap.put(new TopicPartition(topic.name(), partition.partitionIndex()),
Errors.forCode(partition.errorCode()));
errors.add(Errors.forCode(partition.errorCode()));
}
}
return errorCounts(errorMap);
return errorCounts(errors);
}
public static OffsetCommitResponse parse(ByteBuffer buffer, short version) {

View File

@ -17,57 +17,27 @@
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.StopReplicaRequestData;
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionV0;
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopic;
import org.apache.kafka.common.message.StopReplicaResponseData;
import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.FlattenedIterator;
import org.apache.kafka.common.utils.MappedIterator;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
import static org.apache.kafka.common.protocol.types.Type.INT32;
import java.util.stream.Collectors;
public class StopReplicaRequest extends AbstractControlRequest {
private static final Field.Bool DELETE_PARTITIONS = new Field.Bool("delete_partitions", "Boolean which indicates if replica's partitions must be deleted.");
private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions", "The partitions");
private static final Field.Array PARTITION_IDS = new Field.Array("partition_ids", INT32, "The partition ids of a topic");
private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
TOPIC_NAME,
PARTITION_ID);
private static final Field PARTITIONS_V1 = PARTITIONS.withFields(
TOPIC_NAME,
PARTITION_IDS);
private static final Schema STOP_REPLICA_REQUEST_V0 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
DELETE_PARTITIONS,
PARTITIONS_V0);
// STOP_REPLICA_REQUEST_V1 added a broker_epoch Field. This field specifies the generation of the broker across
// bounces. It also normalizes partitions under each topic.
private static final Schema STOP_REPLICA_REQUEST_V1 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
BROKER_EPOCH,
DELETE_PARTITIONS,
PARTITIONS_V1);
public static Schema[] schemaVersions() {
return new Schema[] {STOP_REPLICA_REQUEST_V0, STOP_REPLICA_REQUEST_V1};
}
public static class Builder extends AbstractControlRequest.Builder<StopReplicaRequest> {
private final boolean deletePartitions;
@ -80,10 +50,31 @@ public class StopReplicaRequest extends AbstractControlRequest {
this.partitions = partitions;
}
@Override
public StopReplicaRequest build(short version) {
return new StopReplicaRequest(controllerId, controllerEpoch, brokerEpoch,
deletePartitions, partitions, version);
StopReplicaRequestData data = new StopReplicaRequestData()
.setControllerId(controllerId)
.setControllerEpoch(controllerEpoch)
.setBrokerEpoch(brokerEpoch)
.setDeletePartitions(deletePartitions);
if (version >= 1) {
Map<String, List<Integer>> topicPartitionsMap = CollectionUtils.groupPartitionsByTopic(partitions);
List<StopReplicaTopic> topics = topicPartitionsMap.entrySet().stream().map(entry ->
new StopReplicaTopic()
.setName(entry.getKey())
.setPartitionIndexes(entry.getValue())
).collect(Collectors.toList());
data.setTopics(topics);
} else {
List<StopReplicaPartitionV0> requestPartitions = partitions.stream().map(tp ->
new StopReplicaPartitionV0()
.setTopicName(tp.topic())
.setPartitionIndex(tp.partition())
).collect(Collectors.toList());
data.setUngroupedPartitions(requestPartitions);
}
return new StopReplicaRequest(data, version);
}
@Override
@ -100,54 +91,37 @@ public class StopReplicaRequest extends AbstractControlRequest {
}
}
private final boolean deletePartitions;
private final Collection<TopicPartition> partitions;
private final StopReplicaRequestData data;
private StopReplicaRequest(int controllerId, int controllerEpoch, long brokerEpoch, boolean deletePartitions,
Collection<TopicPartition> partitions, short version) {
super(ApiKeys.STOP_REPLICA, version, controllerId, controllerEpoch, brokerEpoch);
this.deletePartitions = deletePartitions;
this.partitions = partitions;
private StopReplicaRequest(StopReplicaRequestData data, short version) {
super(ApiKeys.STOP_REPLICA, version);
this.data = data;
}
public StopReplicaRequest(Struct struct, short version) {
super(ApiKeys.STOP_REPLICA, struct, version);
partitions = new HashSet<>();
if (version > 0) { // V1
for (Object topicObj : struct.get(PARTITIONS)) {
Struct topicData = (Struct) topicObj;
String topic = topicData.get(TOPIC_NAME);
for (Object partitionObj : topicData.get(PARTITION_IDS)) {
int partition = (Integer) partitionObj;
partitions.add(new TopicPartition(topic, partition));
}
}
} else { // V0
for (Object partitionDataObj : struct.get(PARTITIONS)) {
Struct partitionData = (Struct) partitionDataObj;
String topic = partitionData.get(TOPIC_NAME);
int partition = partitionData.get(PARTITION_ID);
partitions.add(new TopicPartition(topic, partition));
}
}
deletePartitions = struct.get(DELETE_PARTITIONS);
this(new StopReplicaRequestData(struct, version), version);
}
@Override
public StopReplicaResponse getErrorResponse(int throttleTimeMs, Throwable e) {
Errors error = Errors.forException(e);
Map<TopicPartition, Errors> responses = new HashMap<>(partitions.size());
for (TopicPartition partition : partitions) {
responses.put(partition, error);
StopReplicaResponseData data = new StopReplicaResponseData();
data.setErrorCode(error.code());
List<StopReplicaPartitionError> partitions = new ArrayList<>();
for (TopicPartition tp : partitions()) {
partitions.add(new StopReplicaPartitionError()
.setTopicName(tp.topic())
.setPartitionIndex(tp.partition())
.setErrorCode(error.code()));
}
data.setPartitionErrors(partitions);
short versionId = version();
switch (versionId) {
case 0:
case 1:
return new StopReplicaResponse(error, responses);
return new StopReplicaResponse(data);
default:
throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
versionId, this.getClass().getSimpleName(), ApiKeys.STOP_REPLICA.latestVersion()));
@ -155,11 +129,40 @@ public class StopReplicaRequest extends AbstractControlRequest {
}
public boolean deletePartitions() {
return deletePartitions;
return data.deletePartitions();
}
public Collection<TopicPartition> partitions() {
return partitions;
/**
* Note that this method has allocation overhead per iterated element, so callers should copy the result into
* another collection if they need to iterate more than once.
*
* Implementation note: we should strive to avoid allocation overhead per element, see
* `UpdateMetadataRequest.partitionStates()` for the preferred approach. That's not possible in this case and
* StopReplicaRequest should be relatively rare in comparison to other request types.
*/
public Iterable<TopicPartition> partitions() {
if (version() >= 1) {
return () -> new FlattenedIterator<>(data.topics().iterator(), topic ->
new MappedIterator<>(topic.partitionIndexes().iterator(), partition ->
new TopicPartition(topic.name(), partition)));
}
return () -> new MappedIterator<>(data.ungroupedPartitions().iterator(),
partition -> new TopicPartition(partition.topicName(), partition.partitionIndex()));
}
@Override
public int controllerId() {
return data.controllerId();
}
@Override
public int controllerEpoch() {
return data.controllerEpoch();
}
@Override
public long brokerEpoch() {
return data.brokerEpoch();
}
public static StopReplicaRequest parse(ByteBuffer buffer, short version) {
@ -168,34 +171,11 @@ public class StopReplicaRequest extends AbstractControlRequest {
@Override
protected Struct toStruct() {
Struct struct = new Struct(ApiKeys.STOP_REPLICA.requestSchema(version()));
struct.set(CONTROLLER_ID, controllerId);
struct.set(CONTROLLER_EPOCH, controllerEpoch);
struct.setIfExists(BROKER_EPOCH, brokerEpoch);
struct.set(DELETE_PARTITIONS, deletePartitions);
if (version() > 0) { // V1
Map<String, List<Integer>> topicPartitionsMap = CollectionUtils.groupPartitionsByTopic(partitions);
List<Struct> topicsData = new ArrayList<>(topicPartitionsMap.size());
for (Map.Entry<String, List<Integer>> entry : topicPartitionsMap.entrySet()) {
Struct topicData = struct.instance(PARTITIONS);
topicData.set(TOPIC_NAME, entry.getKey());
topicData.set(PARTITION_IDS, entry.getValue().toArray());
topicsData.add(topicData);
}
struct.set(PARTITIONS, topicsData.toArray());
} else { // V0
List<Struct> partitionDatas = new ArrayList<>(partitions.size());
for (TopicPartition partition : partitions) {
Struct partitionData = struct.instance(PARTITIONS);
partitionData.set(TOPIC_NAME, partition.topic());
partitionData.set(PARTITION_ID, partition.partition());
partitionDatas.add(partitionData);
}
struct.set(PARTITIONS, partitionDatas.toArray());
}
return struct;
return data.toStruct(version());
}
protected long size() {
return data.size(version());
}
}

View File

@ -16,43 +16,19 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.StopReplicaResponseData;
import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
import java.util.stream.Collectors;
public class StopReplicaResponse extends AbstractResponse {
private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions", "Response for the requests partitions");
private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
TOPIC_NAME,
PARTITION_ID,
ERROR_CODE);
private static final Schema STOP_REPLICA_RESPONSE_V0 = new Schema(
ERROR_CODE,
PARTITIONS_V0);
private static final Schema STOP_REPLICA_RESPONSE_V1 = STOP_REPLICA_RESPONSE_V0;
public static Schema[] schemaVersions() {
return new Schema[] {STOP_REPLICA_RESPONSE_V0, STOP_REPLICA_RESPONSE_V1};
}
private final Map<TopicPartition, Errors> responses;
/**
* Possible error code:
@ -60,71 +36,44 @@ public class StopReplicaResponse extends AbstractResponse {
* STALE_CONTROLLER_EPOCH (11)
* STALE_BROKER_EPOCH (77)
*/
private final Errors error;
private final StopReplicaResponseData data;
public StopReplicaResponse(Errors error, Map<TopicPartition, Errors> responses) {
this.responses = responses;
this.error = error;
public StopReplicaResponse(StopReplicaResponseData data) {
this.data = data;
}
public StopReplicaResponse(Struct struct) {
responses = new HashMap<>();
for (Object responseDataObj : struct.get(PARTITIONS)) {
Struct responseData = (Struct) responseDataObj;
String topic = responseData.get(TOPIC_NAME);
int partition = responseData.get(PARTITION_ID);
Errors error = Errors.forCode(responseData.get(ERROR_CODE));
responses.put(new TopicPartition(topic, partition), error);
}
error = Errors.forCode(struct.get(ERROR_CODE));
public StopReplicaResponse(Struct struct, short version) {
data = new StopReplicaResponseData(struct, version);
}
public Map<TopicPartition, Errors> responses() {
return responses;
public List<StopReplicaPartitionError> partitionErrors() {
return data.partitionErrors();
}
public Errors error() {
return error;
return Errors.forCode(data.errorCode());
}
@Override
public Map<Errors, Integer> errorCounts() {
if (error != Errors.NONE)
if (data.errorCode() != Errors.NONE.code())
// Minor optimization since the top-level error applies to all partitions
return Collections.singletonMap(error, responses.size());
return errorCounts(responses);
return Collections.singletonMap(error(), data.partitionErrors().size());
return errorCounts(data.partitionErrors().stream().map(p -> Errors.forCode(p.errorCode())).collect(Collectors.toList()));
}
public static StopReplicaResponse parse(ByteBuffer buffer, short version) {
return new StopReplicaResponse(ApiKeys.STOP_REPLICA.parseResponse(version, buffer));
return new StopReplicaResponse(ApiKeys.STOP_REPLICA.parseResponse(version, buffer), version);
}
@Override
protected Struct toStruct(short version) {
Struct struct = new Struct(ApiKeys.STOP_REPLICA.responseSchema(version));
List<Struct> responseDatas = new ArrayList<>(responses.size());
for (Map.Entry<TopicPartition, Errors> response : responses.entrySet()) {
Struct partitionData = struct.instance(PARTITIONS);
TopicPartition partition = response.getKey();
partitionData.set(TOPIC_NAME, partition.topic());
partitionData.set(PARTITION_ID, partition.partition());
partitionData.set(ERROR_CODE, response.getValue().code());
responseDatas.add(partitionData);
}
struct.set(PARTITIONS, responseDatas.toArray());
struct.set(ERROR_CODE, error.code());
return struct;
return data.toStruct(version);
}
@Override
public String toString() {
return "StopReplicaResponse(" +
"responses=" + responses +
", error=" + error +
")";
return data.toString();
}
}

View File

@ -88,7 +88,7 @@ public class TxnOffsetCommitResponse extends AbstractResponse {
@Override
public Map<Errors, Integer> errorCounts() {
return errorCounts(errors());
return errorCounts(errors().values());
}
public Map<TopicPartition, Errors> errors() {

View File

@ -16,177 +16,38 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.UpdateMetadataRequestData;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataTopicState;
import org.apache.kafka.common.message.UpdateMetadataResponseData;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Field;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.FlattenedIterator;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
import static org.apache.kafka.common.protocol.types.Type.INT32;
import static java.util.Collections.singletonList;
public class UpdateMetadataRequest extends AbstractControlRequest {
private static final Field.ComplexArray TOPIC_STATES = new Field.ComplexArray("topic_states", "Topic states");
private static final Field.ComplexArray PARTITION_STATES = new Field.ComplexArray("partition_states", "Partition states");
private static final Field.ComplexArray LIVE_BROKERS = new Field.ComplexArray("live_brokers", "Live broekrs");
// PartitionState fields
private static final Field.Int32 LEADER = new Field.Int32("leader", "The broker id for the leader.");
private static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The leader epoch.");
private static final Field.Array ISR = new Field.Array("isr", INT32, "The in sync replica ids.");
private static final Field.Int32 ZK_VERSION = new Field.Int32("zk_version", "The ZK version.");
private static final Field.Array REPLICAS = new Field.Array("replicas", INT32, "The replica ids.");
private static final Field.Array OFFLINE_REPLICAS = new Field.Array("offline_replicas", INT32, "The offline replica ids");
// Live brokers fields
private static final Field.Int32 BROKER_ID = new Field.Int32("id", "The broker id");
private static final Field.ComplexArray ENDPOINTS = new Field.ComplexArray("end_points", "The endpoints");
private static final Field.NullableStr RACK = new Field.NullableStr("rack", "The rack");
// EndPoint fields
private static final Field.Str HOST = new Field.Str("host", "The hostname of the broker.");
private static final Field.Int32 PORT = new Field.Int32("port", "The port on which the broker accepts requests.");
private static final Field.Str LISTENER_NAME = new Field.Str("listener_name", "The listener name.");
private static final Field.Int16 SECURITY_PROTOCOL_TYPE = new Field.Int16("security_protocol_type", "The security protocol type.");
private static final Field PARTITION_STATES_V0 = PARTITION_STATES.withFields(
TOPIC_NAME,
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS);
// PARTITION_STATES_V4 added a per-partition offline_replicas field. This field specifies
// the list of replicas that are offline.
private static final Field PARTITION_STATES_V4 = PARTITION_STATES.withFields(
TOPIC_NAME,
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS,
OFFLINE_REPLICAS);
private static final Field PARTITION_STATES_V5 = PARTITION_STATES.withFields(
PARTITION_ID,
CONTROLLER_EPOCH,
LEADER,
LEADER_EPOCH,
ISR,
ZK_VERSION,
REPLICAS,
OFFLINE_REPLICAS);
// TOPIC_STATES_V5 normalizes TOPIC_STATES_V4 to
// make it more memory efficient
private static final Field TOPIC_STATES_V5 = TOPIC_STATES.withFields(
TOPIC_NAME,
PARTITION_STATES_V5);
// for some reason, V1 sends `port` before `host` while V0 sends `host` before `port
private static final Field ENDPOINTS_V1 = ENDPOINTS.withFields(
PORT,
HOST,
SECURITY_PROTOCOL_TYPE);
private static final Field ENDPOINTS_V3 = ENDPOINTS.withFields(
PORT,
HOST,
LISTENER_NAME,
SECURITY_PROTOCOL_TYPE);
private static final Field LIVE_BROKERS_V0 = LIVE_BROKERS.withFields(
BROKER_ID,
HOST,
PORT);
private static final Field LIVE_BROKERS_V1 = LIVE_BROKERS.withFields(
BROKER_ID,
ENDPOINTS_V1);
private static final Field LIVE_BROKERS_V2 = LIVE_BROKERS.withFields(
BROKER_ID,
ENDPOINTS_V1,
RACK);
private static final Field LIVE_BROKERS_V3 = LIVE_BROKERS.withFields(
BROKER_ID,
ENDPOINTS_V3,
RACK);
private static final Schema UPDATE_METADATA_REQUEST_V0 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V0,
LIVE_BROKERS_V0);
private static final Schema UPDATE_METADATA_REQUEST_V1 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V0,
LIVE_BROKERS_V1);
private static final Schema UPDATE_METADATA_REQUEST_V2 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V0,
LIVE_BROKERS_V2);
private static final Schema UPDATE_METADATA_REQUEST_V3 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V0,
LIVE_BROKERS_V3);
// UPDATE_METADATA_REQUEST_V4 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
private static final Schema UPDATE_METADATA_REQUEST_V4 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
PARTITION_STATES_V4,
LIVE_BROKERS_V3);
// UPDATE_METADATA_REQUEST_V5 added a broker_epoch Field. This field specifies the generation of the broker across
// bounces. It also normalizes partitions under each topic.
private static final Schema UPDATE_METADATA_REQUEST_V5 = new Schema(
CONTROLLER_ID,
CONTROLLER_EPOCH,
BROKER_EPOCH,
TOPIC_STATES_V5,
LIVE_BROKERS_V3);
public static Schema[] schemaVersions() {
return new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, UPDATE_METADATA_REQUEST_V2,
UPDATE_METADATA_REQUEST_V3, UPDATE_METADATA_REQUEST_V4, UPDATE_METADATA_REQUEST_V5};
}
public static class Builder extends AbstractControlRequest.Builder<UpdateMetadataRequest> {
private final Map<TopicPartition, PartitionState> partitionStates;
private final Set<Broker> liveBrokers;
private final List<UpdateMetadataPartitionState> partitionStates;
private final List<UpdateMetadataBroker> liveBrokers;
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
Map<TopicPartition, PartitionState> partitionStates, Set<Broker> liveBrokers) {
List<UpdateMetadataPartitionState> partitionStates, List<UpdateMetadataBroker> liveBrokers) {
super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch);
this.partitionStates = partitionStates;
this.liveBrokers = liveBrokers;
@ -194,14 +55,53 @@ public class UpdateMetadataRequest extends AbstractControlRequest {
@Override
public UpdateMetadataRequest build(short version) {
if (version == 0) {
for (Broker broker : liveBrokers) {
if (broker.endPoints.size() != 1 || broker.endPoints.get(0).securityProtocol != SecurityProtocol.PLAINTEXT) {
throw new UnsupportedVersionException("UpdateMetadataRequest v0 only handles PLAINTEXT endpoints");
if (version < 3) {
for (UpdateMetadataBroker broker : liveBrokers) {
if (version == 0) {
if (broker.endpoints().size() != 1)
throw new UnsupportedVersionException("UpdateMetadataRequest v0 requires a single endpoint");
if (broker.endpoints().get(0).securityProtocol() != SecurityProtocol.PLAINTEXT.id)
throw new UnsupportedVersionException("UpdateMetadataRequest v0 only handles PLAINTEXT endpoints");
// Don't null out `endpoints` since it's ignored by the generated code if version >= 1
UpdateMetadataEndpoint endpoint = broker.endpoints().get(0);
broker.setV0Host(endpoint.host());
broker.setV0Port(endpoint.port());
} else {
if (broker.endpoints().stream().anyMatch(endpoint -> !endpoint.listener().isEmpty() &&
!endpoint.listener().equals(listenerNameFromSecurityProtocol(endpoint)))) {
throw new UnsupportedVersionException("UpdateMetadataRequest v0-v3 does not support custom " +
"listeners, request version: " + version + ", endpoints: " + broker.endpoints());
}
}
}
}
return new UpdateMetadataRequest(version, controllerId, controllerEpoch, brokerEpoch, partitionStates, liveBrokers);
UpdateMetadataRequestData data = new UpdateMetadataRequestData()
.setControllerId(controllerId)
.setControllerEpoch(controllerEpoch)
.setBrokerEpoch(brokerEpoch)
.setLiveBrokers(liveBrokers);
if (version >= 5) {
Map<String, UpdateMetadataTopicState> topicStatesMap = groupByTopic(partitionStates);
data.setTopicStates(new ArrayList<>(topicStatesMap.values()));
} else {
data.setUngroupedPartitionStates(partitionStates);
}
return new UpdateMetadataRequest(data, version);
}
private static Map<String, UpdateMetadataTopicState> groupByTopic(List<UpdateMetadataPartitionState> partitionStates) {
Map<String, UpdateMetadataTopicState> topicStates = new HashMap<>();
for (UpdateMetadataPartitionState partition : partitionStates) {
// We don't null out the topic name in UpdateMetadataTopicState since it's ignored by the generated
// code if version >= 5
UpdateMetadataTopicState topicState = topicStates.computeIfAbsent(partition.topicName(),
t -> new UpdateMetadataTopicState().setTopicName(partition.topicName()));
topicState.partitionStates().add(partition);
}
return topicStates;
}
@Override
@ -218,275 +118,110 @@ public class UpdateMetadataRequest extends AbstractControlRequest {
}
}
public static final class PartitionState {
public final BasePartitionState basePartitionState;
public final List<Integer> offlineReplicas;
private final UpdateMetadataRequestData data;
public PartitionState(int controllerEpoch,
int leader,
int leaderEpoch,
List<Integer> isr,
int zkVersion,
List<Integer> replicas,
List<Integer> offlineReplicas) {
this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas);
this.offlineReplicas = offlineReplicas;
}
private PartitionState(Struct struct) {
int controllerEpoch = struct.get(CONTROLLER_EPOCH);
int leader = struct.get(LEADER);
int leaderEpoch = struct.get(LEADER_EPOCH);
Object[] isrArray = struct.get(ISR);
List<Integer> isr = new ArrayList<>(isrArray.length);
for (Object r : isrArray)
isr.add((Integer) r);
int zkVersion = struct.get(ZK_VERSION);
Object[] replicasArray = struct.get(REPLICAS);
List<Integer> replicas = new ArrayList<>(replicasArray.length);
for (Object r : replicasArray)
replicas.add((Integer) r);
this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas);
this.offlineReplicas = new ArrayList<>();
if (struct.hasField(OFFLINE_REPLICAS)) {
Object[] offlineReplicasArray = struct.get(OFFLINE_REPLICAS);
for (Object r : offlineReplicasArray)
offlineReplicas.add((Integer) r);
}
}
@Override
public String toString() {
return "PartitionState(controllerEpoch=" + basePartitionState.controllerEpoch +
", leader=" + basePartitionState.leader +
", leaderEpoch=" + basePartitionState.leaderEpoch +
", isr=" + Arrays.toString(basePartitionState.isr.toArray()) +
", zkVersion=" + basePartitionState.zkVersion +
", replicas=" + Arrays.toString(basePartitionState.replicas.toArray()) +
", offlineReplicas=" + Arrays.toString(offlineReplicas.toArray()) + ")";
}
private void setStruct(Struct struct) {
struct.set(CONTROLLER_EPOCH, basePartitionState.controllerEpoch);
struct.set(LEADER, basePartitionState.leader);
struct.set(LEADER_EPOCH, basePartitionState.leaderEpoch);
struct.set(ISR, basePartitionState.isr.toArray());
struct.set(ZK_VERSION, basePartitionState.zkVersion);
struct.set(REPLICAS, basePartitionState.replicas.toArray());
struct.setIfExists(OFFLINE_REPLICAS, offlineReplicas.toArray());
}
UpdateMetadataRequest(UpdateMetadataRequestData data, short version) {
super(ApiKeys.UPDATE_METADATA, version);
this.data = data;
// Do this from the constructor to make it thread-safe (even though it's only needed when some methods are called)
normalize();
}
public static final class Broker {
public final int id;
public final List<EndPoint> endPoints;
public final String rack; // introduced in V2
public Broker(int id, List<EndPoint> endPoints, String rack) {
this.id = id;
this.endPoints = endPoints;
this.rack = rack;
}
@Override
public String toString() {
StringBuilder bld = new StringBuilder();
bld.append("(id=").append(id);
bld.append(", endPoints=").append(Utils.join(endPoints, ","));
bld.append(", rack=").append(rack);
bld.append(")");
return bld.toString();
}
}
public static final class EndPoint {
public final String host;
public final int port;
public final SecurityProtocol securityProtocol;
public final ListenerName listenerName; // introduced in V3
public EndPoint(String host, int port, SecurityProtocol securityProtocol, ListenerName listenerName) {
this.host = host;
this.port = port;
this.securityProtocol = securityProtocol;
this.listenerName = listenerName;
}
@Override
public String toString() {
return "(host=" + host + ", port=" + port + ", listenerName=" + listenerName +
", securityProtocol=" + securityProtocol + ")";
}
}
private final Map<TopicPartition, PartitionState> partitionStates;
private final Set<Broker> liveBrokers;
private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, long brokerEpoch,
Map<TopicPartition, PartitionState> partitionStates, Set<Broker> liveBrokers) {
super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch);
this.partitionStates = partitionStates;
this.liveBrokers = liveBrokers;
}
public UpdateMetadataRequest(Struct struct, short versionId) {
super(ApiKeys.UPDATE_METADATA, struct, versionId);
Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
if (struct.hasField(TOPIC_STATES)) {
for (Object topicStatesDataObj : struct.get(TOPIC_STATES)) {
Struct topicStatesData = (Struct) topicStatesDataObj;
String topic = topicStatesData.get(TOPIC_NAME);
for (Object partitionStateDataObj : topicStatesData.get(PARTITION_STATES)) {
Struct partitionStateData = (Struct) partitionStateDataObj;
int partition = partitionStateData.get(PARTITION_ID);
PartitionState partitionState = new PartitionState(partitionStateData);
partitionStates.put(new TopicPartition(topic, partition), partitionState);
private void normalize() {
// Version 0 only supported a single host and port and the protocol was always plaintext
// Version 1 added support for multiple endpoints, each with its own security protocol
// Version 2 added support for rack
// Version 3 added support for listener name, which we can infer from the security protocol for older versions
if (version() < 3) {
for (UpdateMetadataBroker liveBroker : data.liveBrokers()) {
// Set endpoints so that callers can rely on it always being present
if (version() == 0 && liveBroker.endpoints().isEmpty()) {
SecurityProtocol securityProtocol = SecurityProtocol.PLAINTEXT;
liveBroker.setEndpoints(singletonList(new UpdateMetadataEndpoint()
.setHost(liveBroker.v0Host())
.setPort(liveBroker.v0Port())
.setSecurityProtocol(securityProtocol.id)
.setListener(ListenerName.forSecurityProtocol(securityProtocol).value())));
} else {
for (UpdateMetadataEndpoint endpoint : liveBroker.endpoints()) {
// Set listener so that callers can rely on it always being present
if (endpoint.listener().isEmpty())
endpoint.setListener(listenerNameFromSecurityProtocol(endpoint));
}
}
}
} else {
for (Object partitionStateDataObj : struct.get(PARTITION_STATES)) {
Struct partitionStateData = (Struct) partitionStateDataObj;
String topic = partitionStateData.get(TOPIC_NAME);
int partition = partitionStateData.get(PARTITION_ID);
PartitionState partitionState = new PartitionState(partitionStateData);
partitionStates.put(new TopicPartition(topic, partition), partitionState);
}
}
Set<Broker> liveBrokers = new HashSet<>();
for (Object brokerDataObj : struct.get(LIVE_BROKERS)) {
Struct brokerData = (Struct) brokerDataObj;
int brokerId = brokerData.get(BROKER_ID);
// V0
if (brokerData.hasField(HOST)) {
String host = brokerData.get(HOST);
int port = brokerData.get(PORT);
List<EndPoint> endPoints = new ArrayList<>(1);
SecurityProtocol securityProtocol = SecurityProtocol.PLAINTEXT;
endPoints.add(new EndPoint(host, port, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol)));
liveBrokers.add(new Broker(brokerId, endPoints, null));
} else { // V1, V2 or V3
List<EndPoint> endPoints = new ArrayList<>();
for (Object endPointDataObj : brokerData.get(ENDPOINTS)) {
Struct endPointData = (Struct) endPointDataObj;
int port = endPointData.get(PORT);
String host = endPointData.get(HOST);
short protocolTypeId = endPointData.get(SECURITY_PROTOCOL_TYPE);
SecurityProtocol securityProtocol = SecurityProtocol.forId(protocolTypeId);
String listenerName;
if (endPointData.hasField(LISTENER_NAME)) // V3
listenerName = endPointData.get(LISTENER_NAME);
else
listenerName = securityProtocol.name;
endPoints.add(new EndPoint(host, port, securityProtocol, new ListenerName(listenerName)));
if (version() >= 5) {
for (UpdateMetadataTopicState topicState : data.topicStates()) {
for (UpdateMetadataPartitionState partitionState : topicState.partitionStates()) {
// Set the topic name so that we can always present the ungrouped view to callers
partitionState.setTopicName(topicState.topicName());
}
String rack = null;
if (brokerData.hasField(RACK)) { // V2
rack = brokerData.get(RACK);
}
liveBrokers.add(new Broker(brokerId, endPoints, rack));
}
}
this.partitionStates = partitionStates;
this.liveBrokers = liveBrokers;
}
private static String listenerNameFromSecurityProtocol(UpdateMetadataEndpoint endpoint) {
SecurityProtocol securityProtocol = SecurityProtocol.forId(endpoint.securityProtocol());
return ListenerName.forSecurityProtocol(securityProtocol).value();
}
public UpdateMetadataRequest(Struct struct, short version) {
this(new UpdateMetadataRequestData(struct, version), version);
}
@Override
public int controllerId() {
return data.controllerId();
}
@Override
public int controllerEpoch() {
return data.controllerEpoch();
}
@Override
public long brokerEpoch() {
return data.brokerEpoch();
}
@Override
public UpdateMetadataResponse getErrorResponse(int throttleTimeMs, Throwable e) {
short version = version();
if (version <= 5)
return new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.forException(e).code()));
else
throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
version, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA.latestVersion()));
}
public Iterable<UpdateMetadataPartitionState> partitionStates() {
if (version() >= 5) {
return () -> new FlattenedIterator<>(data.topicStates().iterator(),
topicState -> topicState.partitionStates().iterator());
}
return data.ungroupedPartitionStates();
}
public List<UpdateMetadataBroker> liveBrokers() {
return data.liveBrokers();
}
@Override
protected Struct toStruct() {
short version = version();
Struct struct = new Struct(ApiKeys.UPDATE_METADATA.requestSchema(version));
struct.set(CONTROLLER_ID, controllerId);
struct.set(CONTROLLER_EPOCH, controllerEpoch);
struct.setIfExists(BROKER_EPOCH, brokerEpoch);
if (struct.hasField(TOPIC_STATES)) {
Map<String, Map<Integer, PartitionState>> topicStates = CollectionUtils.groupPartitionDataByTopic(partitionStates);
List<Struct> topicStatesData = new ArrayList<>(topicStates.size());
for (Map.Entry<String, Map<Integer, PartitionState>> entry : topicStates.entrySet()) {
Struct topicStateData = struct.instance(TOPIC_STATES);
topicStateData.set(TOPIC_NAME, entry.getKey());
Map<Integer, PartitionState> partitionMap = entry.getValue();
List<Struct> partitionStatesData = new ArrayList<>(partitionMap.size());
for (Map.Entry<Integer, PartitionState> partitionEntry : partitionMap.entrySet()) {
Struct partitionStateData = topicStateData.instance(PARTITION_STATES);
partitionStateData.set(PARTITION_ID, partitionEntry.getKey());
partitionEntry.getValue().setStruct(partitionStateData);
partitionStatesData.add(partitionStateData);
}
topicStateData.set(PARTITION_STATES, partitionStatesData.toArray());
topicStatesData.add(topicStateData);
}
struct.set(TOPIC_STATES, topicStatesData.toArray());
} else {
List<Struct> partitionStatesData = new ArrayList<>(partitionStates.size());
for (Map.Entry<TopicPartition, PartitionState> entry : partitionStates.entrySet()) {
Struct partitionStateData = struct.instance(PARTITION_STATES);
TopicPartition topicPartition = entry.getKey();
partitionStateData.set(TOPIC_NAME, topicPartition.topic());
partitionStateData.set(PARTITION_ID, topicPartition.partition());
entry.getValue().setStruct(partitionStateData);
partitionStatesData.add(partitionStateData);
}
struct.set(PARTITION_STATES, partitionStatesData.toArray());
}
List<Struct> brokersData = new ArrayList<>(liveBrokers.size());
for (Broker broker : liveBrokers) {
Struct brokerData = struct.instance(LIVE_BROKERS);
brokerData.set(BROKER_ID, broker.id);
if (version == 0) {
EndPoint endPoint = broker.endPoints.get(0);
brokerData.set(HOST, endPoint.host);
brokerData.set(PORT, endPoint.port);
} else {
List<Struct> endPointsData = new ArrayList<>(broker.endPoints.size());
for (EndPoint endPoint : broker.endPoints) {
Struct endPointData = brokerData.instance(ENDPOINTS);
endPointData.set(PORT, endPoint.port);
endPointData.set(HOST, endPoint.host);
endPointData.set(SECURITY_PROTOCOL_TYPE, endPoint.securityProtocol.id);
if (version >= 3)
endPointData.set(LISTENER_NAME, endPoint.listenerName.value());
endPointsData.add(endPointData);
}
brokerData.set(ENDPOINTS, endPointsData.toArray());
if (version >= 2) {
brokerData.set(RACK, broker.rack);
}
}
brokersData.add(brokerData);
}
struct.set(LIVE_BROKERS, brokersData.toArray());
return struct;
return data.toStruct(version());
}
@Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
short versionId = version();
if (versionId <= 5)
return new UpdateMetadataResponse(Errors.forException(e));
else
throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
versionId, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA.latestVersion()));
protected ByteBuffer toBytes() {
ByteBuffer bytes = ByteBuffer.allocate(size());
data.write(new ByteBufferAccessor(bytes), version());
bytes.flip();
return bytes;
}
public Map<TopicPartition, PartitionState> partitionStates() {
return partitionStates;
}
public Set<Broker> liveBrokers() {
return liveBrokers;
protected int size() {
return data.size(version());
}
public static UpdateMetadataRequest parse(ByteBuffer buffer, short version) {

View File

@ -16,62 +16,41 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.UpdateMetadataResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.Map;
import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
public class UpdateMetadataResponse extends AbstractResponse {
private static final Schema UPDATE_METADATA_RESPONSE_V0 = new Schema(ERROR_CODE);
private static final Schema UPDATE_METADATA_RESPONSE_V1 = UPDATE_METADATA_RESPONSE_V0;
private static final Schema UPDATE_METADATA_RESPONSE_V2 = UPDATE_METADATA_RESPONSE_V1;
private static final Schema UPDATE_METADATA_RESPONSE_V3 = UPDATE_METADATA_RESPONSE_V2;
private static final Schema UPDATE_METADATA_RESPONSE_V4 = UPDATE_METADATA_RESPONSE_V3;
private static final Schema UPDATE_METADATA_RESPONSE_V5 = UPDATE_METADATA_RESPONSE_V4;
public static Schema[] schemaVersions() {
return new Schema[]{UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, UPDATE_METADATA_RESPONSE_V2,
UPDATE_METADATA_RESPONSE_V3, UPDATE_METADATA_RESPONSE_V4, UPDATE_METADATA_RESPONSE_V5};
private final UpdateMetadataResponseData data;
public UpdateMetadataResponse(UpdateMetadataResponseData data) {
this.data = data;
}
/**
* Possible error code:
*
* STALE_CONTROLLER_EPOCH (11)
* STALE_BROKER_EPOCH (77)
*/
private final Errors error;
public UpdateMetadataResponse(Errors error) {
this.error = error;
}
public UpdateMetadataResponse(Struct struct) {
error = Errors.forCode(struct.get(ERROR_CODE));
public UpdateMetadataResponse(Struct struct, short version) {
this(new UpdateMetadataResponseData(struct, version));
}
public Errors error() {
return error;
return Errors.forCode(data.errorCode());
}
@Override
public Map<Errors, Integer> errorCounts() {
return errorCounts(error);
return errorCounts(error());
}
public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) {
return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA.parseResponse(version, buffer));
return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA.parseResponse(version, buffer), version);
}
@Override
protected Struct toStruct(short version) {
Struct struct = new Struct(ApiKeys.UPDATE_METADATA.responseSchema(version));
struct.set(ERROR_CODE, error.code());
return struct;
return data.toStruct(version);
}
}
}

View File

@ -0,0 +1,45 @@
/*
* 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.utils;
import java.util.Iterator;
import java.util.function.Function;
/**
* Provides a flattened iterator over the inner elements of an outer iterator.
*/
public final class FlattenedIterator<O, I> extends AbstractIterator<I> {
private final Iterator<O> outerIterator;
private final Function<O, Iterator<I>> innerIteratorFunction;
private Iterator<I> innerIterator;
public FlattenedIterator(Iterator<O> outerIterator, Function<O, Iterator<I>> innerIteratorFunction) {
this.outerIterator = outerIterator;
this.innerIteratorFunction = innerIteratorFunction;
}
@Override
public I makeNext() {
while (innerIterator == null || !innerIterator.hasNext()) {
if (outerIterator.hasNext())
innerIterator = innerIteratorFunction.apply(outerIterator.next());
else
return allDone();
}
return innerIterator.next();
}
}

View File

@ -0,0 +1,44 @@
/*
* 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.utils;
import java.util.Iterator;
import java.util.function.Function;
/**
* An iterator that maps another iterator's elements from type `F` to type `T`.
*/
public final class MappedIterator<F, T> implements Iterator<T> {
private final Iterator<? extends F> underlyingIterator;
private final Function<F, T> mapper;
public MappedIterator(Iterator<? extends F> underlyingIterator, Function<F, T> mapper) {
this.underlyingIterator = underlyingIterator;
this.mapper = mapper;
}
@Override
public final boolean hasNext() {
return underlyingIterator.hasNext();
}
@Override
public final T next() {
return mapper.apply(underlyingIterator.next());
}
}

View File

@ -30,16 +30,16 @@
"about": "The current controller epoch." },
{ "name": "BrokerEpoch", "type": "int64", "versions": "2+", "ignorable": true, "default": "-1",
"about": "The current broker epoch." },
{ "name": "PartitionStatesV0", "type": "[]LeaderAndIsrRequestPartition", "versions": "0-1",
{ "name": "UngroupedPartitionStates", "type": "[]LeaderAndIsrPartitionState", "versions": "0-1",
"about": "The state of each partition, in a v0 or v1 message." },
// In v0 or v1 requests, each partition is listed alongside its topic name.
// In v2+ requests, partitions are organized by topic, so that each topic name
// only needs to be listed once.
{ "name": "TopicStates", "type": "[]LeaderAndIsrRequestTopicState", "versions": "2+",
{ "name": "TopicStates", "type": "[]LeaderAndIsrTopicState", "versions": "2+",
"about": "Each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "2+", "entityType": "topicName",
{ "name": "TopicName", "type": "string", "versions": "2+", "entityType": "topicName",
"about": "The topic name." },
{ "name": "PartitionStatesV0", "type": "[]LeaderAndIsrRequestPartition", "versions": "2+",
{ "name": "PartitionStates", "type": "[]LeaderAndIsrPartitionState", "versions": "2+",
"about": "The state of each partition" }
]},
{ "name": "LiveLeaders", "type": "[]LeaderAndIsrLiveLeader", "versions": "0+",
@ -53,18 +53,18 @@
]}
],
"commonStructs": [
{ "name": "LeaderAndIsrRequestPartition", "versions": "0+", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0-1", "entityType": "topicName",
{ "name": "LeaderAndIsrPartitionState", "versions": "0+", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0-1", "entityType": "topicName", "ignorable": true,
"about": "The topic name. This is only present in v0 or v1." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
"about": "The controller epoch." },
{ "name": "LeaderKey", "type": "int32", "versions": "0+", "entityType": "brokerId",
{ "name": "Leader", "type": "int32", "versions": "0+", "entityType": "brokerId",
"about": "The broker ID of the leader." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The leader epoch." },
{ "name": "IsrReplicas", "type": "[]int32", "versions": "0+",
{ "name": "Isr", "type": "[]int32", "versions": "0+",
"about": "The in-sync replica IDs." },
{ "name": "ZkVersion", "type": "int32", "versions": "0+",
"about": "The ZooKeeper version." },

View File

@ -26,7 +26,7 @@
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "Partitions", "type": "[]LeaderAndIsrResponsePartition", "versions": "0+",
{ "name": "PartitionErrors", "type": "[]LeaderAndIsrPartitionError", "versions": "0+",
"about": "Each partition.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The topic name." },

View File

@ -29,14 +29,14 @@
"about": "The broker epoch." },
{ "name": "DeletePartitions", "type": "bool", "versions": "0+",
"about": "Whether these partitions should be deleted." },
{ "name": "PartitionsV0", "type": "[]StopReplicaRequestPartitionV0", "versions": "0",
{ "name": "UngroupedPartitions", "type": "[]StopReplicaPartitionV0", "versions": "0",
"about": "The partitions to stop.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0", "entityType": "topicName",
"about": "The topic name." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0",
"about": "The partition index." }
]},
{ "name": "Topics", "type": "[]StopReplicaRequestTopic", "versions": "1+",
{ "name": "Topics", "type": "[]StopReplicaTopic", "versions": "1+",
"about": "The topics to stop.", "fields": [
{ "name": "Name", "type": "string", "versions": "1+", "entityType": "topicName",
"about": "The topic name." },

View File

@ -22,7 +22,7 @@
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The top-level error code, or 0 if there was no top-level error." },
{ "name": "Partitions", "type": "[]StopReplicaResponsePartition", "versions": "0+",
{ "name": "PartitionErrors", "type": "[]StopReplicaPartitionError", "versions": "0+",
"about": "The responses for each partition.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The topic name." },

View File

@ -34,16 +34,16 @@
"about": "The controller epoch." },
{ "name": "BrokerEpoch", "type": "int64", "versions": "5+", "ignorable": true, "default": "-1",
"about": "The broker epoch." },
{ "name": "LegacyPartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "0-4",
{ "name": "UngroupedPartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "0-4",
"about": "In older versions of this RPC, each partition that we would like to update." },
{ "name": "TopicStates", "type": "[]UpdateMetadataRequestTopicState", "versions": "5+",
{ "name": "TopicStates", "type": "[]UpdateMetadataTopicState", "versions": "5+",
"about": "In newer versions of this RPC, each topic that we would like to update.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "5+", "entityType": "topicName",
"about": "The topic name." },
{ "name": "PartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "5+",
"about": "The partition that we would like to update." }
]},
{ "name": "Brokers", "type": "[]UpdateMetadataRequestBroker", "versions": "0+", "fields": [
{ "name": "LiveBrokers", "type": "[]UpdateMetadataBroker", "versions": "0+", "fields": [
{ "name": "Id", "type": "int32", "versions": "0+", "entityType": "brokerId",
"about": "The broker id." },
// Version 0 of the protocol only allowed specifying a single host and
@ -52,13 +52,13 @@
"about": "The broker hostname." },
{ "name": "V0Port", "type": "int32", "versions": "0", "ignorable": true,
"about": "The broker port." },
{ "name": "Endpoints", "type": "[]UpdateMetadataRequestEndpoint", "versions": "1+",
{ "name": "Endpoints", "type": "[]UpdateMetadataEndpoint", "versions": "1+", "ignorable": true,
"about": "The broker endpoints.", "fields": [
{ "name": "Port", "type": "int32", "versions": "1+",
"about": "The port of this endpoint" },
{ "name": "Host", "type": "string", "versions": "1+",
"about": "The hostname of this endpoint" },
{ "name": "Listener", "type": "string", "versions": "3+",
{ "name": "Listener", "type": "string", "versions": "3+", "ignorable": true,
"about": "The listener name." },
{ "name": "SecurityProtocol", "type": "int16", "versions": "1+",
"about": "The security protocol type." }
@ -69,7 +69,7 @@
],
"commonStructs": [
{ "name": "UpdateMetadataPartitionState", "versions": "0+", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0-4", "entityType": "topicName",
{ "name": "TopicName", "type": "string", "versions": "0-4", "entityType": "topicName", "ignorable": true,
"about": "In older versions of this RPC, the topic name." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
@ -85,7 +85,7 @@
"about": "The Zookeeper version." },
{ "name": "Replicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
"about": "All the replicas of this partition." },
{ "name": "OfflineReplicas", "type": "[]int32", "versions": "4+", "entityType": "brokerId",
{ "name": "OfflineReplicas", "type": "[]int32", "versions": "4+", "entityType": "brokerId", "ignorable": true,
"about": "The replicas of this partition which are offline." }
]}
]

View File

@ -248,33 +248,29 @@ public final class MessageTest {
@Test
public void testLeaderAndIsrVersions() throws Exception {
// Version 3 adds two new fields - AddingReplicas and RemovingReplicas
LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState partitionStateNoAddingRemovingReplicas =
new LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState()
.setName("topic")
.setPartitionStatesV0(
Collections.singletonList(
new LeaderAndIsrRequestData.LeaderAndIsrRequestPartition()
.setPartitionIndex(0)
.setReplicas(Collections.singletonList(0))
)
);
LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState partitionStateWithAddingRemovingReplicas =
new LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState()
.setName("topic")
.setPartitionStatesV0(
Collections.singletonList(
new LeaderAndIsrRequestData.LeaderAndIsrRequestPartition()
.setPartitionIndex(0)
.setReplicas(Collections.singletonList(0))
.setAddingReplicas(Collections.singletonList(1))
.setRemovingReplicas(Collections.singletonList(1))
)
);
LeaderAndIsrRequestData.LeaderAndIsrTopicState partitionStateNoAddingRemovingReplicas =
new LeaderAndIsrRequestData.LeaderAndIsrTopicState()
.setTopicName("topic")
.setPartitionStates(Collections.singletonList(
new LeaderAndIsrRequestData.LeaderAndIsrPartitionState()
.setPartitionIndex(0)
.setReplicas(Collections.singletonList(0))
));
LeaderAndIsrRequestData.LeaderAndIsrTopicState partitionStateWithAddingRemovingReplicas =
new LeaderAndIsrRequestData.LeaderAndIsrTopicState()
.setTopicName("topic")
.setPartitionStates(Collections.singletonList(
new LeaderAndIsrRequestData.LeaderAndIsrPartitionState()
.setPartitionIndex(0)
.setReplicas(Collections.singletonList(0))
.setAddingReplicas(Collections.singletonList(1))
.setRemovingReplicas(Collections.singletonList(1))
));
testAllMessageRoundTripsBetweenVersions(
(short) 2,
(short) 3,
new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas)),
new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateNoAddingRemovingReplicas)));
(short) 2,
(short) 3,
new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas)),
new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateNoAddingRemovingReplicas)));
testAllMessageRoundTripsFromVersion((short) 3, new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas)));
}

View File

@ -1,87 +0,0 @@
/*
* 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.requests;
import org.apache.kafka.common.TopicPartition;
import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
import java.util.Set;
import java.util.Map;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Random;
public class ControlRequestTest {
@Test
public void testLeaderAndIsrRequestNormalization() {
Set<TopicPartition> tps = generateRandomTopicPartitions(10, 10);
Map<TopicPartition, LeaderAndIsrRequest.PartitionState> partitionStates = new HashMap<>();
for (TopicPartition tp: tps) {
partitionStates.put(tp, new LeaderAndIsrRequest.PartitionState(0, 0, 0,
Collections.emptyList(), 0, Collections.emptyList(), false));
}
LeaderAndIsrRequest.Builder builder = new LeaderAndIsrRequest.Builder((short) 2, 0, 0, 0,
partitionStates, Collections.emptySet());
Assert.assertTrue(builder.build((short) 2).size() < builder.build((short) 1).size());
}
@Test
public void testUpdateMetadataRequestNormalization() {
Set<TopicPartition> tps = generateRandomTopicPartitions(10, 10);
Map<TopicPartition, UpdateMetadataRequest.PartitionState> partitionStates = new HashMap<>();
for (TopicPartition tp: tps) {
partitionStates.put(tp, new UpdateMetadataRequest.PartitionState(0, 0, 0,
Collections.emptyList(), 0, Collections.emptyList(), Collections.emptyList()));
}
UpdateMetadataRequest.Builder builder = new UpdateMetadataRequest.Builder((short) 5, 0, 0, 0,
partitionStates, Collections.emptySet());
Assert.assertTrue(builder.build((short) 5).size() < builder.build((short) 4).size());
}
@Test
public void testStopReplicaRequestNormalization() {
Set<TopicPartition> tps = generateRandomTopicPartitions(10, 10);
Map<TopicPartition, UpdateMetadataRequest.PartitionState> partitionStates = new HashMap<>();
for (TopicPartition tp: tps) {
partitionStates.put(tp, new UpdateMetadataRequest.PartitionState(0, 0, 0,
Collections.emptyList(), 0, Collections.emptyList(), Collections.emptyList()));
}
StopReplicaRequest.Builder builder = new StopReplicaRequest.Builder((short) 5, 0, 0, 0, false, tps);
Assert.assertTrue(builder.build((short) 1).size() < builder.build((short) 0).size());
}
private Set<TopicPartition> generateRandomTopicPartitions(int numTopic, int numPartitionPerTopic) {
Set<TopicPartition> tps = new HashSet<>();
Random r = new Random();
for (int i = 0; i < numTopic; i++) {
byte[] array = new byte[32];
r.nextBytes(array);
String topic = new String(array);
for (int j = 0; j < numPartitionPerTopic; j++) {
tps.add(new TopicPartition(topic, j));
}
}
return tps;
}
}

View File

@ -0,0 +1,147 @@
/*
* 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.requests;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.LeaderAndIsrRequestData;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrLiveLeader;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.apache.kafka.common.protocol.ApiKeys.LEADER_AND_ISR;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class LeaderAndIsrRequestTest {
/**
* Verifies the logic we have in LeaderAndIsrRequest to present a unified interface across the various versions
* works correctly. For example, `LeaderAndIsrPartitionState.topicName` is not serialiazed/deserialized in
* recent versions, but we set it manually so that we can always present the ungrouped partition states
* independently of the version.
*/
@Test
public void testVersionLogic() {
for (short version = LEADER_AND_ISR.oldestVersion(); version <= LEADER_AND_ISR.latestVersion(); version++) {
List<LeaderAndIsrPartitionState> partitionStates = asList(
new LeaderAndIsrPartitionState()
.setTopicName("topic0")
.setPartitionIndex(0)
.setControllerEpoch(2)
.setLeader(0)
.setLeaderEpoch(10)
.setIsr(asList(0, 1))
.setZkVersion(10)
.setReplicas(asList(0, 1, 2))
.setAddingReplicas(asList(3))
.setRemovingReplicas(asList(2)),
new LeaderAndIsrPartitionState()
.setTopicName("topic0")
.setPartitionIndex(1)
.setControllerEpoch(2)
.setLeader(1)
.setLeaderEpoch(11)
.setIsr(asList(1, 2, 3))
.setZkVersion(11)
.setReplicas(asList(1, 2, 3))
.setAddingReplicas(emptyList())
.setRemovingReplicas(emptyList()),
new LeaderAndIsrPartitionState()
.setTopicName("topic1")
.setPartitionIndex(0)
.setControllerEpoch(2)
.setLeader(2)
.setLeaderEpoch(11)
.setIsr(asList(2, 3, 4))
.setZkVersion(11)
.setReplicas(asList(2, 3, 4))
.setAddingReplicas(emptyList())
.setRemovingReplicas(emptyList())
);
List<Node> liveNodes = asList(
new Node(0, "host0", 9090),
new Node(1, "host1", 9091)
);
LeaderAndIsrRequest request = new LeaderAndIsrRequest.Builder(version, 1, 2, 3, partitionStates,
liveNodes).build();
List<LeaderAndIsrLiveLeader> liveLeaders = liveNodes.stream().map(n -> new LeaderAndIsrLiveLeader()
.setBrokerId(n.id())
.setHostName(n.host())
.setPort(n.port())).collect(Collectors.toList());
assertEquals(new HashSet<>(partitionStates), iterableToSet(request.partitionStates()));
assertEquals(liveLeaders, request.liveLeaders());
assertEquals(1, request.controllerId());
assertEquals(2, request.controllerEpoch());
assertEquals(3, request.brokerEpoch());
ByteBuffer byteBuffer = request.toBytes();
LeaderAndIsrRequest deserializedRequest = new LeaderAndIsrRequest(new LeaderAndIsrRequestData(
new ByteBufferAccessor(byteBuffer), version), version);
// Adding/removing replicas is only supported from version 3, so the deserialized request won't have
// them for earlier versions.
if (version < 3) {
partitionStates.get(0)
.setAddingReplicas(emptyList())
.setRemovingReplicas(emptyList());
}
assertEquals(new HashSet<>(partitionStates), iterableToSet(deserializedRequest.partitionStates()));
assertEquals(liveLeaders, deserializedRequest.liveLeaders());
assertEquals(1, request.controllerId());
assertEquals(2, request.controllerEpoch());
assertEquals(3, request.brokerEpoch());
}
}
@Test
public void testTopicPartitionGroupingSizeReduction() {
Set<TopicPartition> tps = TestUtils.generateRandomTopicPartitions(10, 10);
List<LeaderAndIsrPartitionState> partitionStates = new ArrayList<>();
for (TopicPartition tp : tps) {
partitionStates.add(new LeaderAndIsrPartitionState()
.setTopicName(tp.topic())
.setPartitionIndex(tp.partition()));
}
LeaderAndIsrRequest.Builder builder = new LeaderAndIsrRequest.Builder((short) 2, 0, 0, 0,
partitionStates, Collections.emptySet());
LeaderAndIsrRequest v2 = builder.build((short) 2);
LeaderAndIsrRequest v1 = builder.build((short) 1);
assertTrue("Expected v2 < v1: v2=" + v2.size() + ", v1=" + v1.size(), v2.size() < v1.size());
}
private <T> Set<T> iterableToSet(Iterable<T> iterable) {
return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toSet());
}
}

View File

@ -16,16 +16,19 @@
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -33,32 +36,50 @@ public class LeaderAndIsrResponseTest {
@Test
public void testErrorCountsFromGetErrorResponse() {
HashMap<TopicPartition, LeaderAndIsrRequest.PartitionState> partitionStates = new HashMap<>();
partitionStates.put(new TopicPartition("foo", 0), new LeaderAndIsrRequest.PartitionState(15, 1, 10,
Collections.singletonList(10), 20, Collections.singletonList(10), false));
partitionStates.put(new TopicPartition("foo", 1), new LeaderAndIsrRequest.PartitionState(15, 1, 10,
Collections.singletonList(10), 20, Collections.singletonList(10), false));
List<LeaderAndIsrPartitionState> partitionStates = new ArrayList<>();
partitionStates.add(new LeaderAndIsrPartitionState()
.setTopicName("foo")
.setPartitionIndex(0)
.setControllerEpoch(15)
.setLeader(1)
.setLeaderEpoch(10)
.setIsr(Collections.singletonList(10))
.setZkVersion(20)
.setReplicas(Collections.singletonList(10))
.setIsNew(false));
partitionStates.add(new LeaderAndIsrPartitionState()
.setTopicName("foo")
.setPartitionIndex(1)
.setControllerEpoch(15)
.setLeader(1)
.setLeaderEpoch(10)
.setIsr(Collections.singletonList(10))
.setZkVersion(20)
.setReplicas(Collections.singletonList(10))
.setIsNew(false));
LeaderAndIsrRequest request = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion(),
15, 20, 0, partitionStates, Collections.<Node>emptySet()).build();
15, 20, 0, partitionStates, Collections.emptySet()).build();
LeaderAndIsrResponse response = request.getErrorResponse(0, Errors.CLUSTER_AUTHORIZATION_FAILED.exception());
assertEquals(Collections.singletonMap(Errors.CLUSTER_AUTHORIZATION_FAILED, 2), response.errorCounts());
}
@Test
public void testErrorCountsWithTopLevelError() {
Map<TopicPartition, Errors> errors = new HashMap<>();
errors.put(new TopicPartition("foo", 0), Errors.NONE);
errors.put(new TopicPartition("foo", 1), Errors.NOT_LEADER_FOR_PARTITION);
LeaderAndIsrResponse response = new LeaderAndIsrResponse(Errors.UNKNOWN_SERVER_ERROR, errors);
List<LeaderAndIsrPartitionError> partitions = createPartitions("foo",
asList(Errors.NONE, Errors.NOT_LEADER_FOR_PARTITION));
LeaderAndIsrResponse response = new LeaderAndIsrResponse(new LeaderAndIsrResponseData()
.setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())
.setPartitionErrors(partitions));
assertEquals(Collections.singletonMap(Errors.UNKNOWN_SERVER_ERROR, 2), response.errorCounts());
}
@Test
public void testErrorCountsNoTopLevelError() {
Map<TopicPartition, Errors> errors = new HashMap<>();
errors.put(new TopicPartition("foo", 0), Errors.NONE);
errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED);
LeaderAndIsrResponse response = new LeaderAndIsrResponse(Errors.NONE, errors);
List<LeaderAndIsrPartitionError> partitions = createPartitions("foo",
asList(Errors.NONE, Errors.CLUSTER_AUTHORIZATION_FAILED));
LeaderAndIsrResponse response = new LeaderAndIsrResponse(new LeaderAndIsrResponseData()
.setErrorCode(Errors.NONE.code())
.setPartitionErrors(partitions));
Map<Errors, Integer> errorCounts = response.errorCounts();
assertEquals(2, errorCounts.size());
assertEquals(1, errorCounts.get(Errors.NONE).intValue());
@ -67,14 +88,27 @@ public class LeaderAndIsrResponseTest {
@Test
public void testToString() {
Map<TopicPartition, Errors> errors = new HashMap<>();
errors.put(new TopicPartition("foo", 0), Errors.NONE);
errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED);
LeaderAndIsrResponse response = new LeaderAndIsrResponse(Errors.NONE, errors);
List<LeaderAndIsrPartitionError> partitions = createPartitions("foo",
asList(Errors.NONE, Errors.CLUSTER_AUTHORIZATION_FAILED));
LeaderAndIsrResponse response = new LeaderAndIsrResponse(new LeaderAndIsrResponseData()
.setErrorCode(Errors.NONE.code())
.setPartitionErrors(partitions));
String responseStr = response.toString();
assertTrue(responseStr.contains(LeaderAndIsrResponse.class.getSimpleName()));
assertTrue(responseStr.contains(errors.toString()));
assertTrue(responseStr.contains(Errors.NONE.name()));
assertTrue(responseStr.contains(partitions.toString()));
assertTrue(responseStr.contains("errorCode=" + Errors.NONE.code()));
}
private List<LeaderAndIsrPartitionError> createPartitions(String topicName, List<Errors> errors) {
List<LeaderAndIsrPartitionError> partitions = new ArrayList<>();
int partitionIndex = 0;
for (Errors error : errors) {
partitions.add(new LeaderAndIsrPartitionError()
.setTopicName(topicName)
.setPartitionIndex(partitionIndex++)
.setErrorCode(error.code()));
}
return partitions;
}
}

View File

@ -76,6 +76,8 @@ import org.apache.kafka.common.message.InitProducerIdRequestData;
import org.apache.kafka.common.message.InitProducerIdResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.message.JoinGroupResponseData;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.message.ListGroupsRequestData;
@ -97,7 +99,12 @@ import org.apache.kafka.common.message.SaslAuthenticateRequestData;
import org.apache.kafka.common.message.SaslAuthenticateResponseData;
import org.apache.kafka.common.message.SaslHandshakeRequestData;
import org.apache.kafka.common.message.SaslHandshakeResponseData;
import org.apache.kafka.common.message.StopReplicaResponseData;
import org.apache.kafka.common.message.TxnOffsetCommitRequestData;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState;
import org.apache.kafka.common.message.UpdateMetadataResponseData;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys;
@ -237,6 +244,8 @@ public class RequestResponseTest {
checkErrorResponse(createLeaderAndIsrRequest(0), new UnknownServerException(), false);
checkRequest(createLeaderAndIsrRequest(1), true);
checkErrorResponse(createLeaderAndIsrRequest(1), new UnknownServerException(), false);
checkRequest(createLeaderAndIsrRequest(2), true);
checkErrorResponse(createLeaderAndIsrRequest(2), new UnknownServerException(), false);
checkResponse(createLeaderAndIsrResponse(), 0, true);
checkRequest(createSaslHandshakeRequest(), true);
checkErrorResponse(createSaslHandshakeRequest(), new UnknownServerException(), true);
@ -1115,9 +1124,14 @@ public class RequestResponseTest {
}
private StopReplicaResponse createStopReplicaResponse() {
Map<TopicPartition, Errors> responses = new HashMap<>();
responses.put(new TopicPartition("test", 0), Errors.NONE);
return new StopReplicaResponse(Errors.NONE, responses);
List<StopReplicaResponseData.StopReplicaPartitionError> partitions = new ArrayList<>();
partitions.add(new StopReplicaResponseData.StopReplicaPartitionError()
.setTopicName("test")
.setPartitionIndex(0)
.setErrorCode(Errors.NONE.code()));
return new StopReplicaResponse(new StopReplicaResponseData()
.setErrorCode(Errors.NONE.code())
.setPartitionErrors(partitions));
}
private ControlledShutdownRequest createControlledShutdownRequest() {
@ -1155,15 +1169,39 @@ public class RequestResponseTest {
}
private LeaderAndIsrRequest createLeaderAndIsrRequest(int version) {
Map<TopicPartition, LeaderAndIsrRequest.PartitionState> partitionStates = new HashMap<>();
List<LeaderAndIsrPartitionState> partitionStates = new ArrayList<>();
List<Integer> isr = asList(1, 2);
List<Integer> replicas = asList(1, 2, 3, 4);
partitionStates.put(new TopicPartition("topic5", 105),
new LeaderAndIsrRequest.PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas, false));
partitionStates.put(new TopicPartition("topic5", 1),
new LeaderAndIsrRequest.PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas, false));
partitionStates.put(new TopicPartition("topic20", 1),
new LeaderAndIsrRequest.PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas, false));
partitionStates.add(new LeaderAndIsrPartitionState()
.setTopicName("topic5")
.setPartitionIndex(105)
.setControllerEpoch(0)
.setLeader(2)
.setLeaderEpoch(1)
.setIsr(isr)
.setZkVersion(2)
.setReplicas(replicas)
.setIsNew(false));
partitionStates.add(new LeaderAndIsrPartitionState()
.setTopicName("topic5")
.setPartitionIndex(1)
.setControllerEpoch(1)
.setLeader(1)
.setLeaderEpoch(1)
.setIsr(isr)
.setZkVersion(2)
.setReplicas(replicas)
.setIsNew(false));
partitionStates.add(new LeaderAndIsrPartitionState()
.setTopicName("topic20")
.setPartitionIndex(1)
.setControllerEpoch(1)
.setLeader(0)
.setLeaderEpoch(1)
.setIsr(isr)
.setZkVersion(2)
.setReplicas(replicas)
.setIsNew(false));
Set<Node> leaders = Utils.mkSet(
new Node(0, "test0", 1223),
@ -1173,49 +1211,97 @@ public class RequestResponseTest {
}
private LeaderAndIsrResponse createLeaderAndIsrResponse() {
Map<TopicPartition, Errors> responses = new HashMap<>();
responses.put(new TopicPartition("test", 0), Errors.NONE);
return new LeaderAndIsrResponse(Errors.NONE, responses);
List<LeaderAndIsrResponseData.LeaderAndIsrPartitionError> partitions = new ArrayList<>();
partitions.add(new LeaderAndIsrResponseData.LeaderAndIsrPartitionError()
.setTopicName("test")
.setPartitionIndex(0)
.setErrorCode(Errors.NONE.code()));
return new LeaderAndIsrResponse(new LeaderAndIsrResponseData()
.setErrorCode(Errors.NONE.code())
.setPartitionErrors(partitions));
}
private UpdateMetadataRequest createUpdateMetadataRequest(int version, String rack) {
Map<TopicPartition, UpdateMetadataRequest.PartitionState> partitionStates = new HashMap<>();
List<UpdateMetadataPartitionState> partitionStates = new ArrayList<>();
List<Integer> isr = asList(1, 2);
List<Integer> replicas = asList(1, 2, 3, 4);
List<Integer> offlineReplicas = asList();
partitionStates.put(new TopicPartition("topic5", 105),
new UpdateMetadataRequest.PartitionState(0, 2, 1, isr, 2, replicas, offlineReplicas));
partitionStates.put(new TopicPartition("topic5", 1),
new UpdateMetadataRequest.PartitionState(1, 1, 1, isr, 2, replicas, offlineReplicas));
partitionStates.put(new TopicPartition("topic20", 1),
new UpdateMetadataRequest.PartitionState(1, 0, 1, isr, 2, replicas, offlineReplicas));
partitionStates.add(new UpdateMetadataPartitionState()
.setTopicName("topic5")
.setPartitionIndex(105)
.setControllerEpoch(0)
.setLeader(2)
.setLeaderEpoch(1)
.setIsr(isr)
.setZkVersion(2)
.setReplicas(replicas)
.setOfflineReplicas(offlineReplicas));
partitionStates.add(new UpdateMetadataPartitionState()
.setTopicName("topic5")
.setPartitionIndex(1)
.setControllerEpoch(1)
.setLeader(1)
.setLeaderEpoch(1)
.setIsr(isr)
.setZkVersion(2)
.setReplicas(replicas)
.setOfflineReplicas(offlineReplicas));
partitionStates.add(new UpdateMetadataPartitionState()
.setTopicName("topic20")
.setPartitionIndex(1)
.setControllerEpoch(1)
.setLeader(0)
.setLeaderEpoch(1)
.setIsr(isr)
.setZkVersion(2)
.setReplicas(replicas)
.setOfflineReplicas(offlineReplicas));
SecurityProtocol plaintext = SecurityProtocol.PLAINTEXT;
List<UpdateMetadataRequest.EndPoint> endPoints1 = new ArrayList<>();
endPoints1.add(new UpdateMetadataRequest.EndPoint("host1", 1223, plaintext,
ListenerName.forSecurityProtocol(plaintext)));
List<UpdateMetadataEndpoint> endpoints1 = new ArrayList<>();
endpoints1.add(new UpdateMetadataEndpoint()
.setHost("host1")
.setPort(1223)
.setSecurityProtocol(plaintext.id)
.setListener(ListenerName.forSecurityProtocol(plaintext).value()));
List<UpdateMetadataRequest.EndPoint> endPoints2 = new ArrayList<>();
endPoints2.add(new UpdateMetadataRequest.EndPoint("host1", 1244, plaintext,
ListenerName.forSecurityProtocol(plaintext)));
List<UpdateMetadataEndpoint> endpoints2 = new ArrayList<>();
endpoints2.add(new UpdateMetadataEndpoint()
.setHost("host1")
.setPort(1244)
.setSecurityProtocol(plaintext.id)
.setListener(ListenerName.forSecurityProtocol(plaintext).value()));
if (version > 0) {
SecurityProtocol ssl = SecurityProtocol.SSL;
endPoints2.add(new UpdateMetadataRequest.EndPoint("host2", 1234, ssl,
ListenerName.forSecurityProtocol(ssl)));
endPoints2.add(new UpdateMetadataRequest.EndPoint("host2", 1334, ssl,
new ListenerName("CLIENT")));
endpoints2.add(new UpdateMetadataEndpoint()
.setHost("host2")
.setPort(1234)
.setSecurityProtocol(ssl.id)
.setListener(ListenerName.forSecurityProtocol(ssl).value()));
endpoints2.add(new UpdateMetadataEndpoint()
.setHost("host2")
.setPort(1334)
.setSecurityProtocol(ssl.id));
if (version >= 3)
endpoints2.get(1).setListener("CLIENT");
}
Set<UpdateMetadataRequest.Broker> liveBrokers = Utils.mkSet(
new UpdateMetadataRequest.Broker(0, endPoints1, rack),
new UpdateMetadataRequest.Broker(1, endPoints2, rack)
List<UpdateMetadataBroker> liveBrokers = Arrays.asList(
new UpdateMetadataBroker()
.setId(0)
.setEndpoints(endpoints1)
.setRack(rack),
new UpdateMetadataBroker()
.setId(1)
.setEndpoints(endpoints2)
.setRack(rack)
);
return new UpdateMetadataRequest.Builder((short) version, 1, 10, 0, partitionStates,
liveBrokers).build();
liveBrokers).build();
}
private UpdateMetadataResponse createUpdateMetadataResponse() {
return new UpdateMetadataResponse(Errors.NONE);
return new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.NONE.code()));
}
private SaslHandshakeRequest createSaslHandshakeRequest() {

View File

@ -16,25 +16,21 @@
*/
package org.apache.kafka.common.requests;
import java.util.List;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
// This class contains the common fields shared between LeaderAndIsrRequest.PartitionState and UpdateMetadataRequest.PartitionState
public class BasePartitionState {
import java.util.Set;
public final int controllerEpoch;
public final int leader;
public final int leaderEpoch;
public final List<Integer> isr;
public final int zkVersion;
public final List<Integer> replicas;
import static org.junit.Assert.assertTrue;
BasePartitionState(int controllerEpoch, int leader, int leaderEpoch, List<Integer> isr, int zkVersion, List<Integer> replicas) {
this.controllerEpoch = controllerEpoch;
this.leader = leader;
this.leaderEpoch = leaderEpoch;
this.isr = isr;
this.zkVersion = zkVersion;
this.replicas = replicas;
public class StopReplicaRequestTest {
@Test
public void testStopReplicaRequestNormalization() {
Set<TopicPartition> tps = TestUtils.generateRandomTopicPartitions(10, 10);
StopReplicaRequest.Builder builder = new StopReplicaRequest.Builder((short) 5, 0, 0, 0, false, tps);
assertTrue(builder.build((short) 1).size() < builder.build((short) 0).size());
}
}

View File

@ -17,13 +17,16 @@
package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.StopReplicaResponseData;
import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.utils.Utils;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
@ -41,19 +44,25 @@ public class StopReplicaResponseTest {
@Test
public void testErrorCountsWithTopLevelError() {
Map<TopicPartition, Errors> errors = new HashMap<>();
errors.put(new TopicPartition("foo", 0), Errors.NONE);
errors.put(new TopicPartition("foo", 1), Errors.NOT_LEADER_FOR_PARTITION);
StopReplicaResponse response = new StopReplicaResponse(Errors.UNKNOWN_SERVER_ERROR, errors);
List<StopReplicaPartitionError> errors = new ArrayList<>();
errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(0));
errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(1)
.setErrorCode(Errors.NOT_LEADER_FOR_PARTITION.code()));
StopReplicaResponse response = new StopReplicaResponse(new StopReplicaResponseData()
.setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())
.setPartitionErrors(errors));
assertEquals(Collections.singletonMap(Errors.UNKNOWN_SERVER_ERROR, 2), response.errorCounts());
}
@Test
public void testErrorCountsNoTopLevelError() {
Map<TopicPartition, Errors> errors = new HashMap<>();
errors.put(new TopicPartition("foo", 0), Errors.NONE);
errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED);
StopReplicaResponse response = new StopReplicaResponse(Errors.NONE, errors);
List<StopReplicaPartitionError> errors = new ArrayList<>();
errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(0));
errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(1)
.setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()));
StopReplicaResponse response = new StopReplicaResponse(new StopReplicaResponseData()
.setErrorCode(Errors.NONE.code())
.setPartitionErrors(errors));
Map<Errors, Integer> errorCounts = response.errorCounts();
assertEquals(2, errorCounts.size());
assertEquals(1, errorCounts.get(Errors.NONE).intValue());
@ -62,14 +71,15 @@ public class StopReplicaResponseTest {
@Test
public void testToString() {
Map<TopicPartition, Errors> errors = new HashMap<>();
errors.put(new TopicPartition("foo", 0), Errors.NONE);
errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED);
StopReplicaResponse response = new StopReplicaResponse(Errors.NONE, errors);
List<StopReplicaPartitionError> errors = new ArrayList<>();
errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(0));
errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(1)
.setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()));
StopReplicaResponse response = new StopReplicaResponse(new StopReplicaResponseData().setPartitionErrors(errors));
String responseStr = response.toString();
assertTrue(responseStr.contains(StopReplicaResponse.class.getSimpleName()));
assertTrue(responseStr.contains(errors.toString()));
assertTrue(responseStr.contains(Errors.NONE.name()));
assertTrue(responseStr.contains("errorCode=" + Errors.NONE.code()));
}
}

View File

@ -0,0 +1,187 @@
/*
* 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.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.UpdateMetadataRequestData;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.apache.kafka.common.protocol.ApiKeys.UPDATE_METADATA;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class UpdateMetadataRequestTest {
/**
* Verifies the logic we have in UpdateMetadataRequest to present a unified interface across the various versions
* works correctly. For example, `UpdateMetadataPartitionState.topicName` is not serialiazed/deserialized in
* recent versions, but we set it manually so that we can always present the ungrouped partition states
* independently of the version.
*/
@Test
public void testVersionLogic() {
for (short version = UPDATE_METADATA.oldestVersion(); version <= UPDATE_METADATA.latestVersion(); version++) {
List<UpdateMetadataPartitionState> partitionStates = asList(
new UpdateMetadataPartitionState()
.setTopicName("topic0")
.setPartitionIndex(0)
.setControllerEpoch(2)
.setLeader(0)
.setLeaderEpoch(10)
.setIsr(asList(0, 1))
.setZkVersion(10)
.setReplicas(asList(0, 1, 2))
.setOfflineReplicas(asList(2)),
new UpdateMetadataPartitionState()
.setTopicName("topic0")
.setPartitionIndex(1)
.setControllerEpoch(2)
.setLeader(1)
.setLeaderEpoch(11)
.setIsr(asList(1, 2, 3))
.setZkVersion(11)
.setReplicas(asList(1, 2, 3))
.setOfflineReplicas(emptyList()),
new UpdateMetadataPartitionState()
.setTopicName("topic1")
.setPartitionIndex(0)
.setControllerEpoch(2)
.setLeader(2)
.setLeaderEpoch(11)
.setIsr(asList(2, 3))
.setZkVersion(11)
.setReplicas(asList(2, 3, 4))
.setOfflineReplicas(emptyList())
);
List<UpdateMetadataEndpoint> broker0Endpoints = new ArrayList<>();
broker0Endpoints.add(
new UpdateMetadataEndpoint()
.setHost("host0")
.setPort(9090)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id));
// Non plaintext endpoints are only supported from version 1
if (version >= 1) {
broker0Endpoints.add(new UpdateMetadataEndpoint()
.setHost("host0")
.setPort(9091)
.setSecurityProtocol(SecurityProtocol.SSL.id));
}
// Custom listeners are only supported from version 3
if (version >= 3) {
broker0Endpoints.get(0).setListener("listener0");
broker0Endpoints.get(1).setListener("listener1");
}
List<UpdateMetadataBroker> liveBrokers = asList(
new UpdateMetadataBroker()
.setId(0)
.setRack("rack0")
.setEndpoints(broker0Endpoints),
new UpdateMetadataBroker()
.setId(1)
.setEndpoints(asList(
new UpdateMetadataEndpoint()
.setHost("host1")
.setPort(9090)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener("PLAINTEXT")
))
);
UpdateMetadataRequest request = new UpdateMetadataRequest.Builder(version, 1, 2, 3,
partitionStates, liveBrokers).build();
assertEquals(new HashSet<>(partitionStates), iterableToSet(request.partitionStates()));
assertEquals(liveBrokers, request.liveBrokers());
assertEquals(1, request.controllerId());
assertEquals(2, request.controllerEpoch());
assertEquals(3, request.brokerEpoch());
ByteBuffer byteBuffer = request.toBytes();
UpdateMetadataRequest deserializedRequest = new UpdateMetadataRequest(new UpdateMetadataRequestData(
new ByteBufferAccessor(byteBuffer), version), version);
// Unset fields that are not supported in this version as the deserialized request won't have them
// Rack is only supported from version 2
if (version < 2) {
for (UpdateMetadataBroker liveBroker : liveBrokers)
liveBroker.setRack("");
}
// Non plaintext listener name is only supported from version 3
if (version < 3) {
for (UpdateMetadataBroker liveBroker : liveBrokers) {
for (UpdateMetadataEndpoint endpoint : liveBroker.endpoints()) {
SecurityProtocol securityProtocol = SecurityProtocol.forId(endpoint.securityProtocol());
endpoint.setListener(ListenerName.forSecurityProtocol(securityProtocol).value());
}
}
}
// Offline replicas are only supported from version 4
if (version < 4)
partitionStates.get(0).setOfflineReplicas(emptyList());
assertEquals(new HashSet<>(partitionStates), iterableToSet(deserializedRequest.partitionStates()));
assertEquals(liveBrokers, deserializedRequest.liveBrokers());
assertEquals(1, request.controllerId());
assertEquals(2, request.controllerEpoch());
assertEquals(3, request.brokerEpoch());
}
}
@Test
public void testTopicPartitionGroupingSizeReduction() {
Set<TopicPartition> tps = TestUtils.generateRandomTopicPartitions(10, 10);
List<UpdateMetadataPartitionState> partitionStates = new ArrayList<>();
for (TopicPartition tp : tps) {
partitionStates.add(new UpdateMetadataPartitionState()
.setTopicName(tp.topic())
.setPartitionIndex(tp.partition()));
}
UpdateMetadataRequest.Builder builder = new UpdateMetadataRequest.Builder((short) 5, 0, 0, 0,
partitionStates, Collections.emptyList());
assertTrue(builder.build((short) 5).size() < builder.build((short) 4).size());
}
private <T> Set<T> iterableToSet(Iterable<T> iterable) {
return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toSet());
}
}

View File

@ -0,0 +1,115 @@
/*
* 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.utils;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.junit.Assert.assertEquals;
public class FlattenedIteratorTest {
@Test
public void testNestedLists() {
List<List<String>> list = asList(
asList("foo", "a", "bc"),
asList("ddddd"),
asList("", "bar2", "baz45"));
Iterable<String> flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator());
List<String> flattened = new ArrayList<>();
flattenedIterable.forEach(flattened::add);
assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened);
// Ensure we can iterate multiple times
List<String> flattened2 = new ArrayList<>();
flattenedIterable.forEach(flattened2::add);
assertEquals(flattened, flattened2);
}
@Test
public void testEmptyList() {
List<List<String>> list = emptyList();
Iterable<String> flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator());
List<String> flattened = new ArrayList<>();
flattenedIterable.forEach(flattened::add);
assertEquals(emptyList(), flattened);
}
@Test
public void testNestedSingleEmptyList() {
List<List<String>> list = asList(emptyList());
Iterable<String> flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator());
List<String> flattened = new ArrayList<>();
flattenedIterable.forEach(flattened::add);
assertEquals(emptyList(), flattened);
}
@Test
public void testEmptyListFollowedByNonEmpty() {
List<List<String>> list = asList(
emptyList(),
asList("boo", "b", "de"));
Iterable<String> flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator());
List<String> flattened = new ArrayList<>();
flattenedIterable.forEach(flattened::add);
assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened);
}
@Test
public void testEmptyListInBetweenNonEmpty() {
List<List<String>> list = asList(
asList("aadwdwdw"),
emptyList(),
asList("ee", "aa", "dd"));
Iterable<String> flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator());
List<String> flattened = new ArrayList<>();
flattenedIterable.forEach(flattened::add);
assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened);
}
@Test
public void testEmptyListAtTheEnd() {
List<List<String>> list = asList(
asList("ee", "dd"),
asList("e"),
emptyList());
Iterable<String> flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator());
List<String> flattened = new ArrayList<>();
flattenedIterable.forEach(flattened::add);
assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened);
}
}

View File

@ -0,0 +1,61 @@
/*
* 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.utils;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Function;
import java.util.stream.Collectors;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.junit.Assert.assertEquals;
public class MappedIteratorTest {
@Test
public void testStringToInteger() {
List<String> list = asList("foo", "", "bar2", "baz45");
Function<String, Integer> mapper = s -> s.length();
Iterable<Integer> mappedIterable = () -> new MappedIterator<>(list.iterator(), mapper);
List<Integer> mapped = new ArrayList<>();
mappedIterable.forEach(mapped::add);
assertEquals(list.stream().map(mapper).collect(Collectors.toList()), mapped);
// Ensure that we can iterate a second time
List<Integer> mapped2 = new ArrayList<>();
mappedIterable.forEach(mapped2::add);
assertEquals(mapped, mapped2);
}
@Test
public void testEmptyList() {
List<String> list = emptyList();
Function<String, Integer> mapper = s -> s.length();
Iterable<Integer> mappedIterable = () -> new MappedIterator<>(list.iterator(), mapper);
List<Integer> mapped = new ArrayList<>();
mappedIterable.forEach(mapped::add);
assertEquals(emptyList(), mapped);
}
}

View File

@ -446,6 +446,17 @@ public class TestUtils {
return buffer;
}
public static Set<TopicPartition> generateRandomTopicPartitions(int numTopic, int numPartitionPerTopic) {
Set<TopicPartition> tps = new HashSet<>();
for (int i = 0; i < numTopic; i++) {
String topic = randomString(32);
for (int j = 0; j < numPartitionPerTopic; j++) {
tps.add(new TopicPartition(topic, j));
}
}
return tps;
}
public static void assertFutureError(Future<?> future, Class<? extends Throwable> exceptionClass)
throws InterruptedException {
try {

View File

@ -19,6 +19,7 @@ package kafka.cluster
import com.yammer.metrics.core.Gauge
import java.util.concurrent.locks.ReentrantReadWriteLock
import java.util.{Optional, Properties}
import kafka.api.{ApiVersion, LeaderAndIsr, Request}
import kafka.common.UnexpectedAppendOffsetException
import kafka.controller.KafkaController
@ -31,6 +32,7 @@ import kafka.utils._
import kafka.zk.{AdminZkClient, KafkaZkClient}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors._
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
@ -38,6 +40,7 @@ import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
import org.apache.kafka.common.requests.EpochEndOffset._
import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.Time
import scala.collection.JavaConverters._
import scala.collection.{Map, Seq}
@ -476,29 +479,29 @@ class Partition(val topicPartition: TopicPartition,
* If the leader replica id does not change, return false to indicate the replica manager.
*/
def makeLeader(controllerId: Int,
partitionStateInfo: LeaderAndIsrRequest.PartitionState,
partitionState: LeaderAndIsrPartitionState,
correlationId: Int,
highWatermarkCheckpoints: OffsetCheckpoints): Boolean = {
val (leaderHWIncremented, isNewLeader) = inWriteLock(leaderIsrUpdateLock) {
// record the epoch of the controller that made the leadership decision. This is useful while updating the isr
// to maintain the decision maker controller's epoch in the zookeeper path
controllerEpoch = partitionStateInfo.basePartitionState.controllerEpoch
controllerEpoch = partitionState.controllerEpoch
updateAssignmentAndIsr(
assignment = partitionStateInfo.basePartitionState.replicas.asScala.iterator.map(_.toInt).toSeq,
isr = partitionStateInfo.basePartitionState.isr.asScala.iterator.map(_.toInt).toSet
assignment = partitionState.replicas.asScala.iterator.map(_.toInt).toSeq,
isr = partitionState.isr.asScala.iterator.map(_.toInt).toSet
)
createLogIfNotExists(localBrokerId, partitionStateInfo.isNew, isFutureReplica = false, highWatermarkCheckpoints)
createLogIfNotExists(localBrokerId, partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints)
val leaderLog = localLogOrException
val leaderEpochStartOffset = leaderLog.logEndOffset
info(s"$topicPartition starts at Leader Epoch ${partitionStateInfo.basePartitionState.leaderEpoch} from " +
info(s"$topicPartition starts at Leader Epoch ${partitionState.leaderEpoch} from " +
s"offset $leaderEpochStartOffset. Previous Leader Epoch was: $leaderEpoch")
//We cache the leader epoch here, persisting it only if it's local (hence having a log dir)
leaderEpoch = partitionStateInfo.basePartitionState.leaderEpoch
leaderEpoch = partitionState.leaderEpoch
leaderEpochStartOffsetOpt = Some(leaderEpochStartOffset)
zkVersion = partitionStateInfo.basePartitionState.zkVersion
zkVersion = partitionState.zkVersion
// In the case of successive leader elections in a short time period, a follower may have
// entries in its log from a later epoch than any entry in the new leader's log. In order
@ -546,25 +549,25 @@ class Partition(val topicPartition: TopicPartition,
* replica manager that state is already correct and the become-follower steps can be skipped
*/
def makeFollower(controllerId: Int,
partitionStateInfo: LeaderAndIsrRequest.PartitionState,
partitionState: LeaderAndIsrPartitionState,
correlationId: Int,
highWatermarkCheckpoints: OffsetCheckpoints): Boolean = {
inWriteLock(leaderIsrUpdateLock) {
val newLeaderBrokerId = partitionStateInfo.basePartitionState.leader
val newLeaderBrokerId = partitionState.leader
val oldLeaderEpoch = leaderEpoch
// record the epoch of the controller that made the leadership decision. This is useful while updating the isr
// to maintain the decision maker controller's epoch in the zookeeper path
controllerEpoch = partitionStateInfo.basePartitionState.controllerEpoch
controllerEpoch = partitionState.controllerEpoch
updateAssignmentAndIsr(
assignment = partitionStateInfo.basePartitionState.replicas.asScala.iterator.map(_.toInt).toSeq,
assignment = partitionState.replicas.asScala.iterator.map(_.toInt).toSeq,
isr = Set.empty[Int]
)
createLogIfNotExists(localBrokerId, partitionStateInfo.isNew, isFutureReplica = false, highWatermarkCheckpoints)
createLogIfNotExists(localBrokerId, partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints)
leaderEpoch = partitionStateInfo.basePartitionState.leaderEpoch
leaderEpoch = partitionState.leaderEpoch
leaderEpochStartOffsetOpt = None
zkVersion = partitionStateInfo.basePartitionState.zkVersion
zkVersion = partitionState.zkVersion
if (leaderReplicaIdOpt.contains(newLeaderBrokerId) && leaderEpoch == oldLeaderEpoch) {
false

View File

@ -26,10 +26,11 @@ import kafka.metrics.KafkaMetricsGroup
import kafka.server.KafkaConfig
import kafka.utils._
import org.apache.kafka.clients._
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network._
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests.UpdateMetadataRequest.EndPoint
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.security.auth.SecurityProtocol
@ -341,10 +342,10 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
controllerContext: ControllerContext,
stateChangeLogger: StateChangeLogger) extends Logging {
val controllerId: Int = config.brokerId
val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrRequest.PartitionState]]
val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]]
val stopReplicaRequestMap = mutable.Map.empty[Int, ListBuffer[StopReplicaRequestInfo]]
val updateMetadataRequestBrokerSet = mutable.Set.empty[Int]
val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, UpdateMetadataRequest.PartitionState]
val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, UpdateMetadataPartitionState]
def sendEvent(event: ControllerEvent): Unit
@ -382,15 +383,19 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
brokerIds.filter(_ >= 0).foreach { brokerId =>
val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty)
val alreadyNew = result.get(topicPartition).exists(_.isNew)
result.put(topicPartition, new LeaderAndIsrRequest.PartitionState(leaderIsrAndControllerEpoch.controllerEpoch,
leaderIsrAndControllerEpoch.leaderAndIsr.leader,
leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch,
leaderIsrAndControllerEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava,
leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion,
replicaAssignment.replicas.map(Integer.valueOf).asJava,
replicaAssignment.addingReplicas.map(Integer.valueOf).asJava,
replicaAssignment.removingReplicas.map(Integer.valueOf).asJava,
isNew || alreadyNew))
val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
result.put(topicPartition, new LeaderAndIsrPartitionState()
.setTopicName(topicPartition.topic)
.setPartitionIndex(topicPartition.partition)
.setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch)
.setLeader(leaderAndIsr.leader)
.setLeaderEpoch(leaderAndIsr.leaderEpoch)
.setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava)
.setZkVersion(leaderAndIsr.zkVersion)
.setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava)
.setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava)
.setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava)
.setIsNew(isNew || alreadyNew))
}
addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition))
@ -410,25 +415,24 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
partitions: collection.Set[TopicPartition]): Unit = {
def updateMetadataRequestPartitionInfo(partition: TopicPartition, beingDeleted: Boolean): Unit = {
val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition)
leaderIsrAndControllerEpochOpt match {
case Some(l @ LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) =>
controllerContext.partitionLeadershipInfo.get(partition) match {
case Some(LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) =>
val replicas = controllerContext.partitionReplicaAssignment(partition)
val offlineReplicas = replicas.filter(!controllerContext.isReplicaOnline(_, partition))
val leaderIsrAndControllerEpoch = if (beingDeleted) {
val leaderDuringDelete = LeaderAndIsr.duringDelete(leaderAndIsr.isr)
LeaderIsrAndControllerEpoch(leaderDuringDelete, controllerEpoch)
} else {
l
}
val updatedLeaderAndIsr =
if (beingDeleted) LeaderAndIsr.duringDelete(leaderAndIsr.isr)
else leaderAndIsr
val partitionStateInfo = new UpdateMetadataRequest.PartitionState(leaderIsrAndControllerEpoch.controllerEpoch,
leaderIsrAndControllerEpoch.leaderAndIsr.leader,
leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch,
leaderIsrAndControllerEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava,
leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion,
replicas.map(Integer.valueOf).asJava,
offlineReplicas.map(Integer.valueOf).asJava)
val partitionStateInfo = new UpdateMetadataPartitionState()
.setTopicName(partition.topic)
.setPartitionIndex(partition.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(updatedLeaderAndIsr.leader)
.setLeaderEpoch(updatedLeaderAndIsr.leaderEpoch)
.setIsr(updatedLeaderAndIsr.isr.map(Integer.valueOf).asJava)
.setZkVersion(updatedLeaderAndIsr.zkVersion)
.setReplicas(replicas.map(Integer.valueOf).asJava)
.setOfflineReplicas(offlineReplicas.map(Integer.valueOf).asJava)
updateMetadataRequestPartitionInfoMap.put(partition, partitionStateInfo)
case None =>
@ -450,20 +454,21 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
leaderAndIsrRequestMap.filterKeys(controllerContext.liveOrShuttingDownBrokerIds.contains).foreach {
case (broker, leaderAndIsrPartitionStates) =>
leaderAndIsrPartitionStates.foreach {
case (topicPartition, state) =>
if (stateChangeLog.isTraceEnabled) {
leaderAndIsrPartitionStates.foreach { case (topicPartition, state) =>
val typeOfRequest =
if (broker == state.basePartitionState.leader) "become-leader"
if (broker == state.leader) "become-leader"
else "become-follower"
stateChangeLog.trace(s"Sending $typeOfRequest LeaderAndIsr request $state to broker $broker for partition $topicPartition")
}
}
val leaderIds = leaderAndIsrPartitionStates.map(_._2.basePartitionState.leader).toSet
val leaderIds = leaderAndIsrPartitionStates.map(_._2.leader).toSet
val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map {
_.node(config.interBrokerListenerName)
}
val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker)
val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId, controllerEpoch,
brokerEpoch, leaderAndIsrPartitionStates.asJava, leaders.asJava)
brokerEpoch, leaderAndIsrPartitionStates.values.toBuffer.asJava, leaders.asJava)
sendRequest(broker, leaderAndIsrRequestBuilder, (r: AbstractResponse) => sendEvent(LeaderAndIsrResponseReceived(r, broker)))
}
@ -476,7 +481,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
s"for partition $tp")
}
val partitionStates = Map.empty ++ updateMetadataRequestPartitionInfoMap
val partitionStates = updateMetadataRequestPartitionInfoMap.values.toBuffer
val updateMetadataRequestVersion: Short =
if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 5
else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 4
@ -485,23 +490,31 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
else if (config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1
else 0
val liveBrokers = if (updateMetadataRequestVersion == 0) {
// Version 0 of UpdateMetadataRequest only supports PLAINTEXT.
controllerContext.liveOrShuttingDownBrokers.map { broker =>
val liveBrokers = controllerContext.liveOrShuttingDownBrokers.iterator.map { broker =>
val endpoints = if (updateMetadataRequestVersion == 0) {
// Version 0 of UpdateMetadataRequest only supports PLAINTEXT
val securityProtocol = SecurityProtocol.PLAINTEXT
val listenerName = ListenerName.forSecurityProtocol(securityProtocol)
val node = broker.node(listenerName)
val endPoints = Seq(new EndPoint(node.host, node.port, securityProtocol, listenerName))
new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull)
}
} else {
controllerContext.liveOrShuttingDownBrokers.map { broker =>
val endPoints = broker.endPoints.map { endPoint =>
new UpdateMetadataRequest.EndPoint(endPoint.host, endPoint.port, endPoint.securityProtocol, endPoint.listenerName)
Seq(new UpdateMetadataEndpoint()
.setHost(node.host)
.setPort(node.port)
.setSecurityProtocol(securityProtocol.id)
.setListener(listenerName.value))
} else {
broker.endPoints.map { endpoint =>
new UpdateMetadataEndpoint()
.setHost(endpoint.host)
.setPort(endpoint.port)
.setSecurityProtocol(endpoint.securityProtocol.id)
.setListener(endpoint.listenerName.value)
}
new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull)
}
}
new UpdateMetadataBroker()
.setId(broker.id)
.setEndpoints(endpoints.asJava)
.setRack(broker.rack.orNull)
}.toBuffer
updateMetadataRequestBrokerSet.intersect(controllerContext.liveOrShuttingDownBrokerIds).foreach { broker =>
val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker)
@ -520,9 +533,9 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
def stopReplicaPartitionDeleteResponseCallback(brokerId: Int)(response: AbstractResponse): Unit = {
val stopReplicaResponse = response.asInstanceOf[StopReplicaResponse]
val partitionErrorsForDeletingTopics = stopReplicaResponse.responses.asScala.filterKeys { partition =>
controllerContext.isTopicDeletionInProgress(partition.topic)
}.toMap
val partitionErrorsForDeletingTopics = stopReplicaResponse.partitionErrors.asScala.iterator.filter { pe =>
controllerContext.isTopicDeletionInProgress(pe.topicName)
}.map(pe => new TopicPartition(pe.topicName, pe.partitionIndex) -> Errors.forCode(pe.errorCode)).toMap
if (partitionErrorsForDeletingTopics.nonEmpty)
sendEvent(TopicDeletionStopReplicaResponseReceived(brokerId, stopReplicaResponse.error, partitionErrorsForDeletingTopics))

View File

@ -42,6 +42,7 @@ import org.apache.zookeeper.KeeperException.Code
import scala.collection.JavaConverters._
import scala.collection.{Map, Seq, Set, immutable, mutable}
import scala.collection.mutable.ArrayBuffer
import scala.util.{Failure, Try}
sealed trait ElectionTrigger
@ -1312,12 +1313,17 @@ class KafkaController(val config: KafkaConfig,
return
}
val offlineReplicas = leaderAndIsrResponse.responses.asScala.collect {
case (tp, error) if error == Errors.KAFKA_STORAGE_ERROR => tp
}
val onlineReplicas = leaderAndIsrResponse.responses.asScala.collect {
case (tp, error) if error == Errors.NONE => tp
val offlineReplicas = new ArrayBuffer[TopicPartition]()
val onlineReplicas = new ArrayBuffer[TopicPartition]()
leaderAndIsrResponse.partitions.asScala.foreach { partition =>
val tp = new TopicPartition(partition.topicName, partition.partitionIndex)
if (partition.errorCode == Errors.KAFKA_STORAGE_ERROR.code)
offlineReplicas += tp
else if (partition.errorCode == Errors.NONE.code)
onlineReplicas += tp
}
val previousOfflineReplicas = controllerContext.replicasOnOfflineDirs.getOrElse(brokerId, Set.empty[TopicPartition])
val currentOfflineReplicas = previousOfflineReplicas -- onlineReplicas ++ offlineReplicas
controllerContext.replicasOnOfflineDirs.put(brokerId, currentOfflineReplicas)

View File

@ -188,7 +188,7 @@ class AdminManager(val config: KafkaConfig,
case e: Throwable =>
error(s"Error processing create topic request $topic", e)
CreatePartitionsMetadata(topic.name, Map(), ApiError.fromThrowable(e))
}).toIndexedSeq
}).toBuffer
// 2. if timeout <= 0, validateOnly or no topics can proceed return immediately
if (timeout <= 0 || validateOnly || !metadata.exists(_.error.is(Errors.NONE))) {
@ -204,7 +204,7 @@ class AdminManager(val config: KafkaConfig,
} else {
// 3. else pass the assignments and errors to the delayed operation and set the keys
val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this, responseCallback)
val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toIndexedSeq
val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toBuffer
// try to complete the request immediately, otherwise put it into the purgatory
topicPurgatory.tryCompleteElseWatch(delayedCreate, delayedCreateKeys)
}
@ -345,7 +345,7 @@ class AdminManager(val config: KafkaConfig,
val filteredConfigPairs = configs.filter { case (configName, _) =>
/* Always returns true if configNames is None */
configNames.forall(_.contains(configName))
}.toIndexedSeq
}.toBuffer
val configEntries = filteredConfigPairs.map { case (name, value) => createConfigEntry(name, value) }
new DescribeConfigsResponse.Config(ApiError.NONE, configEntries.asJava)

View File

@ -83,6 +83,6 @@ class DelayedCreatePartitions(delayMs: Long,
private def isMissingLeader(topic: String, partition: Int): Boolean = {
val partitionInfo = adminManager.metadataCache.getPartitionInfo(topic, partition)
partitionInfo.isEmpty || partitionInfo.get.basePartitionState.leader == LeaderAndIsr.NoLeader
partitionInfo.forall(_.leader == LeaderAndIsr.NoLeader)
}
}

View File

@ -50,10 +50,10 @@ class DelayedElectLeader(
override def onComplete(): Unit = {
// This could be called to force complete, so I need the full list of partitions, so I can time them all out.
updateWaiting()
val timedout = waitingPartitions.map {
val timedOut = waitingPartitions.map {
case (tp, _) => tp -> new ApiError(Errors.REQUEST_TIMED_OUT, null)
}.toMap
responseCallback(timedout ++ fullResults)
responseCallback(timedOut ++ fullResults)
}
/**
@ -69,17 +69,14 @@ class DelayedElectLeader(
waitingPartitions.isEmpty && forceComplete()
}
private def updateWaiting() = {
waitingPartitions.foreach { case (tp, leader) =>
val ps = replicaManager.metadataCache.getPartitionInfo(tp.topic, tp.partition)
ps match {
case Some(ps) =>
if (leader == ps.basePartitionState.leader) {
waitingPartitions -= tp
fullResults += tp -> ApiError.NONE
}
case None =>
}
private def updateWaiting(): Unit = {
val metadataCache = replicaManager.metadataCache
val completedPartitions = waitingPartitions.collect {
case (tp, leader) if metadataCache.getPartitionInfo(tp.topic, tp.partition).exists(_.leader == leader) => tp
}
completedPartitions.foreach { tp =>
waitingPartitions -= tp
fullResults += tp -> ApiError.NONE
}
}

View File

@ -48,33 +48,14 @@ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal}
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic
import org.apache.kafka.common.message.CreateTopicsResponseData
import org.apache.kafka.common.message.{AlterPartitionReassignmentsResponseData, CreateTopicsResponseData, DeleteGroupsResponseData, DeleteTopicsResponseData, DescribeGroupsResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData}
import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection}
import org.apache.kafka.common.message.DeleteGroupsResponseData
import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection}
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse}
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData
import org.apache.kafka.common.message.DeleteTopicsResponseData
import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection}
import org.apache.kafka.common.message.DescribeGroupsResponseData
import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult
import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult
import org.apache.kafka.common.message.ExpireDelegationTokenResponseData
import org.apache.kafka.common.message.FindCoordinatorResponseData
import org.apache.kafka.common.message.HeartbeatResponseData
import org.apache.kafka.common.message.InitProducerIdResponseData
import org.apache.kafka.common.message.JoinGroupResponseData
import org.apache.kafka.common.message.LeaveGroupResponseData
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse
import org.apache.kafka.common.message.ListGroupsResponseData
import org.apache.kafka.common.message.OffsetCommitRequestData
import org.apache.kafka.common.message.OffsetCommitResponseData
import org.apache.kafka.common.message.OffsetDeleteResponseData
import org.apache.kafka.common.message.RenewDelegationTokenResponseData
import org.apache.kafka.common.message.SaslAuthenticateResponseData
import org.apache.kafka.common.message.SaslHandshakeResponseData
import org.apache.kafka.common.message.SyncGroupResponseData
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{ListenerName, Send}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
@ -247,7 +228,7 @@ class KafkaApis(val requestChannel: RequestChannel,
// for its previous generation so the broker should skip the stale request.
info("Received stop replica request with broker epoch " +
s"${stopReplicaRequest.brokerEpoch()} smaller than the current broker epoch ${controller.brokerEpoch}")
sendResponseExemptThrottle(request, new StopReplicaResponse(Errors.STALE_BROKER_EPOCH, Map.empty[TopicPartition, Errors].asJava))
sendResponseExemptThrottle(request, new StopReplicaResponse(new StopReplicaResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)))
} else {
val (result, error) = replicaManager.stopReplicas(stopReplicaRequest)
// Clearing out the cache for groups that belong to an offsets topic partition for which this broker was the leader,
@ -261,7 +242,16 @@ class KafkaApis(val requestChannel: RequestChannel,
groupCoordinator.handleGroupEmigration(topicPartition.partition)
}
}
sendResponseExemptThrottle(request, new StopReplicaResponse(error, result.asJava))
def toStopReplicaPartition(tp: TopicPartition, error: Errors) =
new StopReplicaResponseData.StopReplicaPartitionError()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setErrorCode(error.code)
sendResponseExemptThrottle(request, new StopReplicaResponse(new StopReplicaResponseData()
.setErrorCode(error.code)
.setPartitionErrors(result.map { case (tp, error) => toStopReplicaPartition(tp, error) }.toBuffer.asJava)))
}
CoreUtils.swallow(replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads(), this)
@ -272,20 +262,21 @@ class KafkaApis(val requestChannel: RequestChannel,
val updateMetadataRequest = request.body[UpdateMetadataRequest]
authorizeClusterOperation(request, CLUSTER_ACTION)
if (isBrokerEpochStale(updateMetadataRequest.brokerEpoch())) {
if (isBrokerEpochStale(updateMetadataRequest.brokerEpoch)) {
// When the broker restarts very quickly, it is possible for this broker to receive request intended
// for its previous generation so the broker should skip the stale request.
info("Received update metadata request with broker epoch " +
s"${updateMetadataRequest.brokerEpoch()} smaller than the current broker epoch ${controller.brokerEpoch}")
sendResponseExemptThrottle(request, new UpdateMetadataResponse(Errors.STALE_BROKER_EPOCH))
s"${updateMetadataRequest.brokerEpoch} smaller than the current broker epoch ${controller.brokerEpoch}")
sendResponseExemptThrottle(request,
new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)))
} else {
val deletedPartitions = replicaManager.maybeUpdateMetadataCache(correlationId, updateMetadataRequest)
if (deletedPartitions.nonEmpty)
groupCoordinator.handleDeletedPartitions(deletedPartitions)
if (adminManager.hasDelayedTopicOperations) {
updateMetadataRequest.partitionStates.keySet.asScala.map(_.topic).foreach { topic =>
adminManager.tryCompleteDelayedTopicOperations(topic)
updateMetadataRequest.partitionStates.asScala.foreach { partitionState =>
adminManager.tryCompleteDelayedTopicOperations(partitionState.topicName)
}
}
quotas.clientQuotaCallback.foreach { callback =>
@ -296,11 +287,13 @@ class KafkaApis(val requestChannel: RequestChannel,
}
}
if (replicaManager.hasDelayedElectionOperations) {
updateMetadataRequest.partitionStates.asScala.foreach { case (tp, _) =>
updateMetadataRequest.partitionStates.asScala.foreach { partitionState =>
val tp = new TopicPartition(partitionState.topicName, partitionState.partitionIndex)
replicaManager.tryCompleteElection(TopicPartitionOperationKey(tp))
}
}
sendResponseExemptThrottle(request, new UpdateMetadataResponse(Errors.NONE))
sendResponseExemptThrottle(request, new UpdateMetadataResponse(
new UpdateMetadataResponseData().setErrorCode(Errors.NONE.code)))
}
}

View File

@ -28,10 +28,12 @@ import kafka.controller.StateChangeLogger
import kafka.utils.CoreUtils._
import kafka.utils.Logging
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState
import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest}
import org.apache.kafka.common.security.auth.SecurityProtocol
/**
@ -75,14 +77,14 @@ class MetadataCache(brokerId: Int) extends Logging {
snapshot.partitionStates.get(topic).map { partitions =>
partitions.map { case (partitionId, partitionState) =>
val topicPartition = new TopicPartition(topic, partitionId.toInt)
val leaderBrokerId = partitionState.basePartitionState.leader
val leaderEpoch = partitionState.basePartitionState.leaderEpoch
val leaderBrokerId = partitionState.leader
val leaderEpoch = partitionState.leaderEpoch
val maybeLeader = getAliveEndpoint(snapshot, leaderBrokerId, listenerName)
val replicas = partitionState.basePartitionState.replicas.asScala
val replicas = partitionState.replicas.asScala
val replicaInfo = getEndpoints(snapshot, replicas, listenerName, errorUnavailableEndpoints)
val offlineReplicaInfo = getEndpoints(snapshot, partitionState.offlineReplicas.asScala, listenerName, errorUnavailableEndpoints)
val isr = partitionState.basePartitionState.isr.asScala
val isr = partitionState.isr.asScala
val isrInfo = getEndpoints(snapshot, isr, listenerName, errorUnavailableEndpoints)
maybeLeader match {
case None =>
@ -148,7 +150,7 @@ class MetadataCache(brokerId: Int) extends Logging {
snapshot.partitionStates.keySet
}
private def getAllPartitions(snapshot: MetadataSnapshot): Map[TopicPartition, UpdateMetadataRequest.PartitionState] = {
private def getAllPartitions(snapshot: MetadataSnapshot): Map[TopicPartition, UpdateMetadataPartitionState] = {
snapshot.partitionStates.flatMap { case (topic, partitionStates) =>
partitionStates.map { case (partition, state ) => (new TopicPartition(topic, partition.toInt), state) }
}.toMap
@ -166,15 +168,15 @@ class MetadataCache(brokerId: Int) extends Logging {
metadataSnapshot.aliveBrokers.values.toBuffer
}
private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]],
private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]],
topic: String,
partitionId: Int,
stateInfo: UpdateMetadataRequest.PartitionState): Unit = {
stateInfo: UpdateMetadataPartitionState): Unit = {
val infos = partitionStates.getOrElseUpdate(topic, mutable.LongMap())
infos(partitionId) = stateInfo
}
def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataRequest.PartitionState] = {
def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataPartitionState] = {
metadataSnapshot.partitionStates.get(topic).flatMap(_.get(partitionId))
}
@ -184,7 +186,7 @@ class MetadataCache(brokerId: Int) extends Logging {
def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] = {
val snapshot = metadataSnapshot
snapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) map { partitionInfo =>
val leaderId = partitionInfo.basePartitionState.leader
val leaderId = partitionInfo.leader
snapshot.aliveNodes.get(leaderId) match {
case Some(nodeMap) =>
@ -197,8 +199,8 @@ class MetadataCache(brokerId: Int) extends Logging {
def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = {
val snapshot = metadataSnapshot
snapshot.partitionStates.get(tp.topic()).flatMap(_.get(tp.partition())).map { partitionInfo =>
val replicaIds = partitionInfo.basePartitionState.replicas
snapshot.partitionStates.get(tp.topic).flatMap(_.get(tp.partition)).map { partitionInfo =>
val replicaIds = partitionInfo.replicas
replicaIds.asScala
.map(replicaId => replicaId.intValue() -> {
snapshot.aliveBrokers.get(replicaId.longValue()) match {
@ -220,17 +222,17 @@ class MetadataCache(brokerId: Int) extends Logging {
val nodes = snapshot.aliveNodes.map { case (id, nodes) => (id, nodes.get(listenerName).orNull) }
def node(id: Integer): Node = nodes.get(id.toLong).orNull
val partitions = getAllPartitions(snapshot)
.filter { case (_, state) => state.basePartitionState.leader != LeaderAndIsr.LeaderDuringDelete }
.filter { case (_, state) => state.leader != LeaderAndIsr.LeaderDuringDelete }
.map { case (tp, state) =>
new PartitionInfo(tp.topic, tp.partition, node(state.basePartitionState.leader),
state.basePartitionState.replicas.asScala.map(node).toArray,
state.basePartitionState.isr.asScala.map(node).toArray,
new PartitionInfo(tp.topic, tp.partition, node(state.leader),
state.replicas.asScala.map(node).toArray,
state.isr.asScala.map(node).toArray,
state.offlineReplicas.asScala.map(node).toArray)
}
val unauthorizedTopics = Collections.emptySet[String]
val internalTopics = getAllTopics(snapshot).filter(Topic.isInternal).asJava
new Cluster(clusterId, nodes.values.filter(_ != null).toList.asJava,
partitions.toList.asJava,
new Cluster(clusterId, nodes.values.filter(_ != null).toBuffer.asJava,
partitions.toBuffer.asJava,
unauthorizedTopics, internalTopics,
snapshot.controllerId.map(id => node(id)).orNull)
}
@ -252,9 +254,10 @@ class MetadataCache(brokerId: Int) extends Logging {
// move to `AnyRefMap`, which has comparable performance.
val nodes = new java.util.HashMap[ListenerName, Node]
val endPoints = new mutable.ArrayBuffer[EndPoint]
broker.endPoints.asScala.foreach { ep =>
endPoints += EndPoint(ep.host, ep.port, ep.listenerName, ep.securityProtocol)
nodes.put(ep.listenerName, new Node(broker.id, ep.host, ep.port))
broker.endpoints.asScala.foreach { ep =>
val listenerName = new ListenerName(ep.listener)
endPoints += new EndPoint(ep.host, ep.port, listenerName, SecurityProtocol.forId(ep.securityProtocol))
nodes.put(listenerName, new Node(broker.id, ep.host, ep.port))
}
aliveBrokers(broker.id) = Broker(broker.id, endPoints, Option(broker.rack))
aliveNodes(broker.id) = nodes.asScala
@ -266,20 +269,21 @@ class MetadataCache(brokerId: Int) extends Logging {
}
val deletedPartitions = new mutable.ArrayBuffer[TopicPartition]
if (updateMetadataRequest.partitionStates().isEmpty) {
if (!updateMetadataRequest.partitionStates.iterator.hasNext) {
metadataSnapshot = MetadataSnapshot(metadataSnapshot.partitionStates, controllerId, aliveBrokers, aliveNodes)
} else {
//since kafka may do partial metadata updates, we start by copying the previous state
val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]](metadataSnapshot.partitionStates.size)
val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]](metadataSnapshot.partitionStates.size)
metadataSnapshot.partitionStates.foreach { case (topic, oldPartitionStates) =>
val copy = new mutable.LongMap[UpdateMetadataRequest.PartitionState](oldPartitionStates.size)
val copy = new mutable.LongMap[UpdateMetadataPartitionState](oldPartitionStates.size)
copy ++= oldPartitionStates
partitionStates += (topic -> copy)
}
updateMetadataRequest.partitionStates.asScala.foreach { case (tp, info) =>
updateMetadataRequest.partitionStates.asScala.foreach { info =>
val controllerId = updateMetadataRequest.controllerId
val controllerEpoch = updateMetadataRequest.controllerEpoch
if (info.basePartitionState.leader == LeaderAndIsr.LeaderDuringDelete) {
val tp = new TopicPartition(info.topicName, info.partitionIndex)
if (info.leader == LeaderAndIsr.LeaderDuringDelete) {
removePartitionInfo(partitionStates, tp.topic, tp.partition)
stateChangeLogger.trace(s"Deleted partition $tp from metadata cache in response to UpdateMetadata " +
s"request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId")
@ -302,7 +306,8 @@ class MetadataCache(brokerId: Int) extends Logging {
def contains(tp: TopicPartition): Boolean = getPartitionInfo(tp.topic, tp.partition).isDefined
private def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]], topic: String, partitionId: Int): Boolean = {
private def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]],
topic: String, partitionId: Int): Boolean = {
partitionStates.get(topic).exists { infos =>
infos.remove(partitionId)
if (infos.isEmpty) partitionStates.remove(topic)
@ -310,7 +315,7 @@ class MetadataCache(brokerId: Int) extends Logging {
}
}
case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]],
case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]],
controllerId: Option[Int],
aliveBrokers: mutable.LongMap[Broker],
aliveNodes: mutable.LongMap[collection.Map[ListenerName, Node]])

View File

@ -37,6 +37,9 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.Node
import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.LeaderAndIsrResponseData
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors
@ -1144,10 +1147,12 @@ class ReplicaManager(val config: KafkaConfig,
def becomeLeaderOrFollower(correlationId: Int,
leaderAndIsrRequest: LeaderAndIsrRequest,
onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): LeaderAndIsrResponse = {
leaderAndIsrRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) =>
stateChangeLogger.trace(s"Received LeaderAndIsr request $stateInfo " +
s"correlation id $correlationId from controller ${leaderAndIsrRequest.controllerId} " +
s"epoch ${leaderAndIsrRequest.controllerEpoch} for partition $topicPartition")
if (stateChangeLogger.isTraceEnabled) {
leaderAndIsrRequest.partitionStates.asScala.foreach { partitionState =>
stateChangeLogger.trace(s"Received LeaderAndIsr request $partitionState " +
s"correlation id $correlationId from controller ${leaderAndIsrRequest.controllerId} " +
s"epoch ${leaderAndIsrRequest.controllerEpoch}")
}
}
replicaStateChangeLock synchronized {
if (leaderAndIsrRequest.controllerEpoch < controllerEpoch) {
@ -1161,10 +1166,11 @@ class ReplicaManager(val config: KafkaConfig,
controllerEpoch = leaderAndIsrRequest.controllerEpoch
// First check partition's leader epoch
val partitionState = new mutable.HashMap[Partition, LeaderAndIsrRequest.PartitionState]()
val partitionStates = new mutable.HashMap[Partition, LeaderAndIsrPartitionState]()
val newPartitions = new mutable.HashSet[Partition]
leaderAndIsrRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) =>
leaderAndIsrRequest.partitionStates.asScala.foreach { partitionState =>
val topicPartition = new TopicPartition(partitionState.topicName, partitionState.partitionIndex)
val partitionOpt = getPartition(topicPartition) match {
case HostedPartition.Offline =>
stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from " +
@ -1185,16 +1191,16 @@ class ReplicaManager(val config: KafkaConfig,
partitionOpt.foreach { partition =>
val currentLeaderEpoch = partition.getLeaderEpoch
val requestLeaderEpoch = stateInfo.basePartitionState.leaderEpoch
val requestLeaderEpoch = partitionState.leaderEpoch
if (requestLeaderEpoch > currentLeaderEpoch) {
// If the leader epoch is valid record the epoch of the controller that made the leadership decision.
// This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path
if (stateInfo.basePartitionState.replicas.contains(localBrokerId))
partitionState.put(partition, stateInfo)
if (partitionState.replicas.contains(localBrokerId))
partitionStates.put(partition, partitionState)
else {
stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from controller $controllerId with " +
s"correlation id $correlationId epoch $controllerEpoch for partition $topicPartition as itself is not " +
s"in assigned replica list ${stateInfo.basePartitionState.replicas.asScala.mkString(",")}")
s"in assigned replica list ${partitionState.replicas.asScala.mkString(",")}")
responseMap.put(topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
}
} else if (requestLeaderEpoch < currentLeaderEpoch) {
@ -1214,10 +1220,10 @@ class ReplicaManager(val config: KafkaConfig,
}
}
val partitionsTobeLeader = partitionState.filter { case (_, stateInfo) =>
stateInfo.basePartitionState.leader == localBrokerId
val partitionsTobeLeader = partitionStates.filter { case (_, partitionState) =>
partitionState.leader == localBrokerId
}
val partitionsToBeFollower = partitionState -- partitionsTobeLeader.keys
val partitionsToBeFollower = partitionStates -- partitionsTobeLeader.keys
val highWatermarkCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints)
val partitionsBecomeLeader = if (partitionsTobeLeader.nonEmpty)
@ -1244,7 +1250,8 @@ class ReplicaManager(val config: KafkaConfig,
// remove metrics for brokers which are not followers of a topic
leaderTopicSet.diff(followerTopicSet).foreach(brokerTopicStats.removeOldFollowerMetrics)
leaderAndIsrRequest.partitionStates.asScala.keys.foreach { topicPartition =>
leaderAndIsrRequest.partitionStates.asScala.foreach { partitionState =>
val topicPartition = new TopicPartition(partitionState.topicName, partitionState.partitionIndex)
/*
* If there is offline log directory, a Partition object may have been created by getOrCreatePartition()
* before getOrCreateReplica() failed to create local replica due to KafkaStorageException.
@ -1284,7 +1291,15 @@ class ReplicaManager(val config: KafkaConfig,
replicaFetcherManager.shutdownIdleFetcherThreads()
replicaAlterLogDirsManager.shutdownIdleFetcherThreads()
onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower)
new LeaderAndIsrResponse(Errors.NONE, responseMap.asJava)
val responsePartitions = responseMap.iterator.map { case (tp, error) =>
new LeaderAndIsrPartitionError()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setErrorCode(error.code)
}.toBuffer
new LeaderAndIsrResponse(new LeaderAndIsrResponseData()
.setErrorCode(Errors.NONE.code)
.setPartitionErrors(responsePartitions.asJava))
}
}
}
@ -1304,42 +1319,42 @@ class ReplicaManager(val config: KafkaConfig,
*/
private def makeLeaders(controllerId: Int,
controllerEpoch: Int,
partitionState: Map[Partition, LeaderAndIsrRequest.PartitionState],
partitionStates: Map[Partition, LeaderAndIsrPartitionState],
correlationId: Int,
responseMap: mutable.Map[TopicPartition, Errors],
highWatermarkCheckpoints: OffsetCheckpoints): Set[Partition] = {
partitionState.keys.foreach { partition =>
partitionStates.keys.foreach { partition =>
stateChangeLogger.trace(s"Handling LeaderAndIsr request correlationId $correlationId from " +
s"controller $controllerId epoch $controllerEpoch starting the become-leader transition for " +
s"partition ${partition.topicPartition}")
}
for (partition <- partitionState.keys)
for (partition <- partitionStates.keys)
responseMap.put(partition.topicPartition, Errors.NONE)
val partitionsToMakeLeaders = mutable.Set[Partition]()
try {
// First stop fetchers for all the partitions
replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(_.topicPartition))
replicaFetcherManager.removeFetcherForPartitions(partitionStates.keySet.map(_.topicPartition))
// Update the partition information to be the leader
partitionState.foreach{ case (partition, partitionStateInfo) =>
partitionStates.foreach { case (partition, partitionState) =>
try {
if (partition.makeLeader(controllerId, partitionStateInfo, correlationId, highWatermarkCheckpoints)) {
if (partition.makeLeader(controllerId, partitionState, correlationId, highWatermarkCheckpoints)) {
partitionsToMakeLeaders += partition
stateChangeLogger.trace(s"Stopped fetchers as part of become-leader request from " +
s"controller $controllerId epoch $controllerEpoch with correlation id $correlationId for partition ${partition.topicPartition} " +
s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch})")
s"(last update controller epoch ${partitionState.controllerEpoch})")
} else
stateChangeLogger.info(s"Skipped the become-leader state change after marking its " +
s"partition as leader with correlation id $correlationId from controller $controllerId epoch $controllerEpoch for " +
s"partition ${partition.topicPartition} (last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) " +
s"partition ${partition.topicPartition} (last update controller epoch ${partitionState.controllerEpoch}) " +
s"since it is already the leader for the partition.")
} catch {
case e: KafkaStorageException =>
stateChangeLogger.error(s"Skipped the become-leader state change with " +
s"correlation id $correlationId from controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition} " +
s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) since " +
s"(last update controller epoch ${partitionState.controllerEpoch}) since " +
s"the replica for the partition is offline due to disk error $e")
val dirOpt = getLogDir(partition.topicPartition)
error(s"Error while making broker the leader for partition $partition in dir $dirOpt", e)
@ -1349,7 +1364,7 @@ class ReplicaManager(val config: KafkaConfig,
} catch {
case e: Throwable =>
partitionState.keys.foreach { partition =>
partitionStates.keys.foreach { partition =>
stateChangeLogger.error(s"Error while processing LeaderAndIsr request correlationId $correlationId received " +
s"from controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition}", e)
}
@ -1357,7 +1372,7 @@ class ReplicaManager(val config: KafkaConfig,
throw e
}
partitionState.keys.foreach { partition =>
partitionStates.keys.foreach { partition =>
stateChangeLogger.trace(s"Completed LeaderAndIsr request correlationId $correlationId from controller $controllerId " +
s"epoch $controllerEpoch for the become-leader transition for partition ${partition.topicPartition}")
}
@ -1385,14 +1400,14 @@ class ReplicaManager(val config: KafkaConfig,
*/
private def makeFollowers(controllerId: Int,
controllerEpoch: Int,
partitionStates: Map[Partition, LeaderAndIsrRequest.PartitionState],
partitionStates: Map[Partition, LeaderAndIsrPartitionState],
correlationId: Int,
responseMap: mutable.Map[TopicPartition, Errors],
highWatermarkCheckpoints: OffsetCheckpoints) : Set[Partition] = {
partitionStates.foreach { case (partition, partitionState) =>
stateChangeLogger.trace(s"Handling LeaderAndIsr request correlationId $correlationId from controller $controllerId " +
s"epoch $controllerEpoch starting the become-follower transition for partition ${partition.topicPartition} with leader " +
s"${partitionState.basePartitionState.leader}")
s"${partitionState.leader}")
}
for (partition <- partitionStates.keys)
@ -1401,37 +1416,37 @@ class ReplicaManager(val config: KafkaConfig,
val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set()
try {
// TODO: Delete leaders from LeaderAndIsrRequest
partitionStates.foreach { case (partition, partitionStateInfo) =>
val newLeaderBrokerId = partitionStateInfo.basePartitionState.leader
partitionStates.foreach { case (partition, partitionState) =>
val newLeaderBrokerId = partitionState.leader
try {
metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match {
// Only change partition state when the leader is available
case Some(_) =>
if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, highWatermarkCheckpoints))
if (partition.makeFollower(controllerId, partitionState, correlationId, highWatermarkCheckpoints))
partitionsToMakeFollower += partition
else
stateChangeLogger.info(s"Skipped the become-follower state change after marking its partition as " +
s"follower with correlation id $correlationId from controller $controllerId epoch $controllerEpoch " +
s"for partition ${partition.topicPartition} (last update " +
s"controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) " +
s"controller epoch ${partitionState.controllerEpoch}) " +
s"since the new leader $newLeaderBrokerId is the same as the old leader")
case None =>
// The leader broker should always be present in the metadata cache.
// If not, we should record the error message and abort the transition process for this partition
stateChangeLogger.error(s"Received LeaderAndIsrRequest with correlation id $correlationId from " +
s"controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition} " +
s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) " +
s"(last update controller epoch ${partitionState.controllerEpoch}) " +
s"but cannot become follower since the new leader $newLeaderBrokerId is unavailable.")
// Create the local replica even if the leader is unavailable. This is required to ensure that we include
// the partition's high watermark in the checkpoint file (see KAFKA-1647)
partition.createLogIfNotExists(localBrokerId, isNew = partitionStateInfo.isNew, isFutureReplica = false,
partition.createLogIfNotExists(localBrokerId, isNew = partitionState.isNew, isFutureReplica = false,
highWatermarkCheckpoints)
}
} catch {
case e: KafkaStorageException =>
stateChangeLogger.error(s"Skipped the become-follower state change with correlation id $correlationId from " +
s"controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition} " +
s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) with leader " +
s"(last update controller epoch ${partitionState.controllerEpoch}) with leader " +
s"$newLeaderBrokerId since the replica for the partition is offline due to disk error $e")
val dirOpt = getLogDir(partition.topicPartition)
error(s"Error while making broker the follower for partition $partition with leader " +
@ -1444,7 +1459,7 @@ class ReplicaManager(val config: KafkaConfig,
partitionsToMakeFollower.foreach { partition =>
stateChangeLogger.trace(s"Stopped fetchers as part of become-follower request from controller $controllerId " +
s"epoch $controllerEpoch with correlation id $correlationId for partition ${partition.topicPartition} with leader " +
s"${partitionStates(partition).basePartitionState.leader}")
s"${partitionStates(partition).leader}")
}
partitionsToMakeFollower.foreach { partition =>
@ -1456,14 +1471,14 @@ class ReplicaManager(val config: KafkaConfig,
partitionsToMakeFollower.foreach { partition =>
stateChangeLogger.trace(s"Truncated logs and checkpointed recovery boundaries for partition " +
s"${partition.topicPartition} as part of become-follower request with correlation id $correlationId from " +
s"controller $controllerId epoch $controllerEpoch with leader ${partitionStates(partition).basePartitionState.leader}")
s"controller $controllerId epoch $controllerEpoch with leader ${partitionStates(partition).leader}")
}
if (isShuttingDown.get()) {
partitionsToMakeFollower.foreach { partition =>
stateChangeLogger.trace(s"Skipped the adding-fetcher step of the become-follower state " +
s"change with correlation id $correlationId from controller $controllerId epoch $controllerEpoch for " +
s"partition ${partition.topicPartition} with leader ${partitionStates(partition).basePartitionState.leader} " +
s"partition ${partition.topicPartition} with leader ${partitionStates(partition).leader} " +
"since it is shutting down")
}
} else {
@ -1493,7 +1508,7 @@ class ReplicaManager(val config: KafkaConfig,
partitionStates.keys.foreach { partition =>
stateChangeLogger.trace(s"Completed LeaderAndIsr request correlationId $correlationId from controller $controllerId " +
s"epoch $controllerEpoch for the become-follower transition for partition ${partition.topicPartition} with leader " +
s"${partitionStates(partition).basePartitionState.leader}")
s"${partitionStates(partition).leader}")
}
partitionsToMakeFollower
@ -1711,7 +1726,7 @@ class ReplicaManager(val config: KafkaConfig,
if (expectedLeaders.nonEmpty) {
val watchKeys = expectedLeaders.iterator.map {
case (tp, _) => TopicPartitionOperationKey(tp)
}.toIndexedSeq
}.toBuffer
delayedElectLeaderPurgatory.tryCompleteElseWatch(
new DelayedElectLeader(

View File

@ -1656,7 +1656,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
private def getTopicConfigs(topics: Set[String]): Seq[GetDataResponse] = {
val getDataRequests: Seq[GetDataRequest] = topics.iterator.map { topic =>
GetDataRequest(ConfigEntityZNode.path(ConfigType.Topic, topic), ctx = Some(topic))
}.toIndexedSeq
}.toBuffer
retryRequestsUntilConnected(getDataRequests)
}

View File

@ -334,7 +334,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest {
@volatile var thrownException: Option[Throwable] = None
@volatile var receivedMessages = 0
@volatile private var partitionAssignment: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
private val partitionAssignment = mutable.Set[TopicPartition]()
@volatile private var subscriptionChanged = false
private var topicsSubscription = topicsToSubscribe
@ -424,7 +424,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest {
}
// make sure that sum of all partitions to all consumers equals total number of partitions
val totalPartitionsInAssignments = (0 /: assignments) (_ + _.size)
val totalPartitionsInAssignments = assignments.foldLeft(0)(_ + _.size)
if (totalPartitionsInAssignments != partitions.size) {
// either same partitions got assigned to more than one consumer or some
// partitions were not assigned
@ -434,7 +434,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest {
// The above checks could miss the case where one or more partitions were assigned to more
// than one consumer and the same number of partitions were missing from assignments.
// Make sure that all unique assignments are the same as 'partitions'
val uniqueAssignedPartitions = (Set[TopicPartition]() /: assignments) (_ ++ _)
val uniqueAssignedPartitions = assignments.foldLeft(Set.empty[TopicPartition])(_ ++ _)
uniqueAssignedPartitions == partitions
}

View File

@ -1306,8 +1306,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
} catch {
case e: ExecutionException =>
assertTrue(e.getCause.isInstanceOf[UnknownMemberIdException])
case _: Throwable =>
fail("Should have caught exception in getting member future")
case t: Throwable =>
fail(s"Should have caught exception in getting member future: $t")
}
}
@ -1341,8 +1341,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
} catch {
case e: ExecutionException =>
assertTrue(e.getCause.isInstanceOf[UnknownMemberIdException])
case _: Throwable =>
fail("Should have caught exception in getting member future")
case t: Throwable =>
fail(s"Should have caught exception in getting member future: $t")
}
}

View File

@ -22,7 +22,7 @@ import java.time.Duration
import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService}
import kafka.log.LogConfig
import kafka.network.SocketServer
import kafka.security.auth.{ResourceType => AuthResourceType, SimpleAclAuthorizer, Topic}
import kafka.security.auth.{SimpleAclAuthorizer, Topic, ResourceType => AuthResourceType}
import kafka.security.authorizer.AuthorizerUtils.WildcardHost
import kafka.server.{BaseRequestTest, KafkaConfig}
import kafka.utils.TestUtils
@ -49,11 +49,15 @@ import org.apache.kafka.common.message.HeartbeatRequestData
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResource, AlterableConfig, AlterableConfigCollection}
import org.apache.kafka.common.message.JoinGroupRequestData
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData
import org.apache.kafka.common.message.OffsetCommitRequestData
import org.apache.kafka.common.message.SyncGroupRequestData
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, Records, SimpleRecord}
@ -200,8 +204,10 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.HEARTBEAT -> ((resp: HeartbeatResponse) => resp.error),
ApiKeys.LEAVE_GROUP -> ((resp: LeaveGroupResponse) => resp.error),
ApiKeys.DELETE_GROUPS -> ((resp: DeleteGroupsResponse) => resp.get(group)),
ApiKeys.LEADER_AND_ISR -> ((resp: requests.LeaderAndIsrResponse) => resp.responses.asScala.find(_._1 == tp).get._2),
ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => resp.responses.asScala.find(_._1 == tp).get._2),
ApiKeys.LEADER_AND_ISR -> ((resp: requests.LeaderAndIsrResponse) => Errors.forCode(
resp.partitions.asScala.find(p => p.topicName == tp.topic && p.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => Errors.forCode(
resp.partitionErrors.asScala.find(pe => pe.topicName == tp.topic && pe.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.CONTROLLED_SHUTDOWN -> ((resp: requests.ControlledShutdownResponse) => resp.error),
ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => Errors.forCode(resp.data.topics.find(createTopic).errorCode())),
ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => Errors.forCode(resp.data.responses.find(deleteTopic).errorCode())),
@ -353,14 +359,25 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
}
private def createUpdateMetadataRequest = {
val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState(
Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava
val partitionStates = Seq(new UpdateMetadataPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(Int.MaxValue)
.setLeader(brokerId)
.setLeaderEpoch(Int.MaxValue)
.setIsr(List(brokerId).asJava)
.setZkVersion(2)
.setReplicas(Seq(brokerId).asJava)).asJava
val securityProtocol = SecurityProtocol.PLAINTEXT
val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId,
Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol,
ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava
val brokers = Seq(new UpdateMetadataBroker()
.setId(brokerId)
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("localhost")
.setPort(0)
.setSecurityProtocol(securityProtocol.id)
.setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)).asJava
val version = ApiKeys.UPDATE_METADATA.latestVersion
new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, Long.MaxValue, partitionState, brokers).build()
new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, Long.MaxValue, partitionStates, brokers).build()
}
private def createJoinGroupRequest = {
@ -442,7 +459,16 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
private def leaderAndIsrRequest = {
new requests.LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Long.MaxValue,
Map(tp -> new LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, false)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(Int.MaxValue)
.setLeader(brokerId)
.setLeaderEpoch(Int.MaxValue)
.setIsr(List(brokerId).asJava)
.setZkVersion(2)
.setReplicas(Seq(brokerId).asJava)
.setIsNew(false)).asJava,
Set(new Node(brokerId, "localhost", 0)).asJava).build()
}
@ -532,7 +558,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
List(new ListPartitionReassignmentsRequestData.ListPartitionReassignmentsTopics()
.setName(topic)
.setPartitionIndexes(
List(new Integer(tp.partition)).asJava
List(Integer.valueOf(tp.partition)).asJava
)).asJava
)
).build()
@ -569,12 +595,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.CREATE_PARTITIONS -> createPartitionsRequest,
ApiKeys.ADD_PARTITIONS_TO_TXN -> addPartitionsToTxnRequest,
ApiKeys.ADD_OFFSETS_TO_TXN -> addOffsetsToTxnRequest,
// Check StopReplica last since some APIs depend on replica availability
ApiKeys.STOP_REPLICA -> stopReplicaRequest,
ApiKeys.ELECT_LEADERS -> electLeadersRequest,
ApiKeys.INCREMENTAL_ALTER_CONFIGS -> incrementalAlterConfigsRequest,
ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> alterPartitionReassignmentsRequest,
ApiKeys.LIST_PARTITION_REASSIGNMENTS -> listPartitionReassignmentsRequest
ApiKeys.LIST_PARTITION_REASSIGNMENTS -> listPartitionReassignmentsRequest,
// Check StopReplica last since some APIs depend on replica availability
ApiKeys.STOP_REPLICA -> stopReplicaRequest
)
for ((key, request) <- requestKeyToRequest) {

View File

@ -260,7 +260,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
val SslKeystorePasswordVal = "${file:ssl.keystore.password:password}"
val configPrefix = listenerPrefix(SecureExternal)
var brokerConfigs = describeConfig(adminClients.head, servers).entries.asScala
val brokerConfigs = describeConfig(adminClients.head, servers).entries.asScala
// the following are values before updated
assertTrue("Initial value of polling interval", brokerConfigs.find(_.name == TestMetricsReporter.PollingIntervalProp) == None)
assertTrue("Initial value of ssl truststore type", brokerConfigs.find(_.name == configPrefix+KafkaConfig.SslTruststoreTypeProp) == None)

View File

@ -93,8 +93,8 @@ class PreferredReplicaLeaderElectionCommandTest extends ZooKeeperTestHarness wit
debug(s"Starting server $targetServer now that a non-preferred replica is leader")
servers(targetServer).startup()
TestUtils.waitUntilTrue(() => servers.forall { server =>
server.metadataCache.getPartitionInfo(partition.topic(), partition.partition()).exists { partitionState =>
partitionState.basePartitionState.isr.contains(targetServer)
server.metadataCache.getPartitionInfo(partition.topic, partition.partition).exists { partitionState =>
partitionState.isr.contains(targetServer)
}
},
s"Replicas for partition $partition not created")
@ -106,8 +106,7 @@ class PreferredReplicaLeaderElectionCommandTest extends ZooKeeperTestHarness wit
private def awaitLeader(topicPartition: TopicPartition, timeoutMs: Long = test.TestUtils.DEFAULT_MAX_WAIT_MS): Int = {
TestUtils.awaitValue(() => {
servers.head.metadataCache.getPartitionInfo(topicPartition.topic, topicPartition.partition)
.map(_.basePartitionState.leader)
servers.head.metadataCache.getPartitionInfo(topicPartition.topic, topicPartition.partition).map(_.leader)
}, s"Timed out waiting to find current leader of $topicPartition", timeoutMs)
}

View File

@ -32,11 +32,12 @@ import kafka.server.checkpoints.OffsetCheckpoints
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{ApiException, OffsetNotAvailableException, ReplicaNotAvailableException}
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{EpochEndOffset, IsolationLevel, LeaderAndIsrRequest, ListOffsetRequest}
import org.apache.kafka.common.requests.{EpochEndOffset, IsolationLevel, ListOffsetRequest}
import org.junit.{After, Before, Test}
import org.junit.Assert._
import org.mockito.Mockito.{doNothing, mock, when}
@ -482,9 +483,14 @@ class PartitionTest {
new SimpleRecord(20,"k4".getBytes, "v2".getBytes),
new SimpleRecord(21,"k5".getBytes, "v3".getBytes)))
val leaderState = new LeaderAndIsrRequest.PartitionState(
controllerEpoch, leader, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true
)
val leaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true)
assertTrue("Expected first makeLeader() to return 'leader changed'",
partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints))
@ -552,15 +558,27 @@ class PartitionTest {
assertEquals(Right(None), fetchOffsetsForTimestamp(30, Some(IsolationLevel.READ_UNCOMMITTED)))
// Make into a follower
val followerState = new LeaderAndIsrRequest.PartitionState(
controllerEpoch, follower2, leaderEpoch + 1, isr, 4, replicas.map(Int.box).asJava, false
)
val followerState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(follower2)
.setLeaderEpoch(leaderEpoch + 1)
.setIsr(isr)
.setZkVersion(4)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(false)
assertTrue(partition.makeFollower(controllerId, followerState, 1, offsetCheckpoints))
// Back to leader, this resets the startLogOffset for this epoch (to 2), we're now in the fault condition
val newLeaderState = new LeaderAndIsrRequest.PartitionState(
controllerEpoch, leader, leaderEpoch + 2, isr, 5, replicas.map(Int.box).asJava, false
)
val newLeaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch + 2)
.setIsr(isr)
.setZkVersion(5)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(false)
assertTrue(partition.makeLeader(controllerId, newLeaderState, 2, offsetCheckpoints))
// Try to get offsets as a client
@ -633,13 +651,25 @@ class PartitionTest {
if (isLeader) {
assertTrue("Expected become leader transition to succeed",
partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints))
partition.makeLeader(controllerId, new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true), 0, offsetCheckpoints))
assertEquals(leaderEpoch, partition.getLeaderEpoch)
} else {
assertTrue("Expected become follower transition to succeed",
partition.makeFollower(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId + 1,
leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints))
partition.makeFollower(controllerId, new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId + 1)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true), 0, offsetCheckpoints))
assertEquals(leaderEpoch, partition.getLeaderEpoch)
assertEquals(None, partition.leaderLogIfLocal)
}
@ -710,8 +740,14 @@ class PartitionTest {
partition.createLogIfNotExists(brokerId, isNew = false, isFutureReplica = false, offsetCheckpoints)
assertTrue("Expected become leader transition to succeed",
partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints))
partition.makeLeader(controllerId, new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true), 0, offsetCheckpoints))
assertEquals(leaderEpoch, partition.getLeaderEpoch)
val records = createTransactionalRecords(List(
@ -773,19 +809,36 @@ class PartitionTest {
@Test
def testMakeFollowerWithNoLeaderIdChange(): Unit = {
// Start off as follower
var partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 1,
List[Integer](0, 1, 2, brokerId).asJava, 1, List[Integer](0, 1, 2, brokerId).asJava, false)
partition.makeFollower(0, partitionStateInfo, 0, offsetCheckpoints)
var partitionState = new LeaderAndIsrPartitionState()
.setControllerEpoch(0)
.setLeader(1)
.setLeaderEpoch(1)
.setIsr(List[Integer](0, 1, 2, brokerId).asJava)
.setZkVersion(1)
.setReplicas(List[Integer](0, 1, 2, brokerId).asJava)
.setIsNew(false)
partition.makeFollower(0, partitionState, 0, offsetCheckpoints)
// Request with same leader and epoch increases by only 1, do become-follower steps
partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4,
List[Integer](0, 1, 2, brokerId).asJava, 1, List[Integer](0, 1, 2, brokerId).asJava, false)
assertTrue(partition.makeFollower(0, partitionStateInfo, 2, offsetCheckpoints))
partitionState = new LeaderAndIsrPartitionState()
.setControllerEpoch(0)
.setLeader(1)
.setLeaderEpoch(4)
.setIsr(List[Integer](0, 1, 2, brokerId).asJava)
.setZkVersion(1)
.setReplicas(List[Integer](0, 1, 2, brokerId).asJava)
.setIsNew(false)
assertTrue(partition.makeFollower(0, partitionState, 2, offsetCheckpoints))
// Request with same leader and same epoch, skip become-follower steps
partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4,
List[Integer](0, 1, 2, brokerId).asJava, 1, List[Integer](0, 1, 2, brokerId).asJava, false)
assertFalse(partition.makeFollower(0, partitionStateInfo, 2, offsetCheckpoints))
partitionState = new LeaderAndIsrPartitionState()
.setControllerEpoch(0)
.setLeader(1)
.setLeaderEpoch(4)
.setIsr(List[Integer](0, 1, 2, brokerId).asJava)
.setZkVersion(1)
.setReplicas(List[Integer](0, 1, 2, brokerId).asJava)
assertFalse(partition.makeFollower(0, partitionState, 2, offsetCheckpoints))
}
@Test
@ -806,7 +859,14 @@ class PartitionTest {
val batch3 = TestUtils.records(records = List(new SimpleRecord("k6".getBytes, "v1".getBytes),
new SimpleRecord("k7".getBytes, "v2".getBytes)))
val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 1, replicas, true)
val leaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true)
assertTrue("Expected first makeLeader() to return 'leader changed'",
partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints))
assertEquals("Current leader epoch", leaderEpoch, partition.getLeaderEpoch)
@ -834,12 +894,24 @@ class PartitionTest {
assertEquals("Expected leader's HW", lastOffsetOfFirstBatch, partition.log.get.highWatermark)
// current leader becomes follower and then leader again (without any new records appended)
val followerState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, follower2, leaderEpoch + 1, isr, 1,
replicas, false)
val followerState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(follower2)
.setLeaderEpoch(leaderEpoch + 1)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(false)
partition.makeFollower(controllerId, followerState, 1, offsetCheckpoints)
val newLeaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch + 2, isr, 1,
replicas, false)
val newLeaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch + 2)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(false)
assertTrue("Expected makeLeader() to return 'leader changed' after makeFollower()",
partition.makeLeader(controllerEpoch, newLeaderState, 2, offsetCheckpoints))
val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset
@ -904,8 +976,14 @@ class PartitionTest {
})
partition.setLog(log, isFutureLog = false)
val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
leaderEpoch, isr, 1, replicaIds, true)
val leaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicaIds)
.setIsNew(true)
partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints)
partitions += partition
}
@ -989,7 +1067,14 @@ class PartitionTest {
assertFalse(partition.isAtMinIsr)
// Make isr set to only have leader to trigger AtMinIsr (default min isr config is 1)
val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 1, replicas, true)
val leaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true)
partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints)
assertTrue(partition.isAtMinIsr)
}
@ -1012,8 +1097,18 @@ class PartitionTest {
val initializeTimeMs = time.milliseconds()
assertTrue("Expected become leader transition to succeed",
partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints))
partition.makeLeader(
controllerId,
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true),
0,
offsetCheckpoints))
val remoteReplica = partition.getReplica(remoteBrokerId).get
assertEquals(initializeTimeMs, remoteReplica.lastCaughtUpTimeMs)
@ -1065,11 +1160,16 @@ class PartitionTest {
"Expected become leader transition to succeed",
partition.makeLeader(
controllerId,
new LeaderAndIsrRequest.PartitionState(
controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true),
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true),
0,
offsetCheckpoints
)
offsetCheckpoints)
)
assertEquals(Set(brokerId), partition.inSyncReplicaIds)
@ -1122,8 +1222,18 @@ class PartitionTest {
partition.createLogIfNotExists(brokerId, isNew = false, isFutureReplica = false, offsetCheckpoints)
assertTrue("Expected become leader transition to succeed",
partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints))
partition.makeLeader(
controllerId,
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true),
0,
offsetCheckpoints))
assertEquals(Set(brokerId), partition.inSyncReplicaIds)
val remoteReplica = partition.getReplica(remoteBrokerId).get
@ -1170,11 +1280,16 @@ class PartitionTest {
"Expected become leader transition to succeed",
partition.makeLeader(
controllerId,
new LeaderAndIsrRequest.PartitionState(
controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true),
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true),
0,
offsetCheckpoints
)
offsetCheckpoints)
)
assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds)
assertEquals(0L, partition.localLogOrException.highWatermark)
@ -1222,11 +1337,16 @@ class PartitionTest {
"Expected become leader transition to succeed",
partition.makeLeader(
controllerId,
new LeaderAndIsrRequest.PartitionState(
controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true),
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true),
0,
offsetCheckpoints
)
offsetCheckpoints)
)
assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds)
assertEquals(0L, partition.localLogOrException.highWatermark)
@ -1289,11 +1409,16 @@ class PartitionTest {
"Expected become leader transition to succeed",
partition.makeLeader(
controllerId,
new LeaderAndIsrRequest.PartitionState(
controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true),
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true),
0,
offsetCheckpoints
)
offsetCheckpoints)
)
assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds)
assertEquals(0L, partition.localLogOrException.highWatermark)
@ -1339,8 +1464,18 @@ class PartitionTest {
val initializeTimeMs = time.milliseconds()
partition.createLogIfNotExists(brokerId, isNew = false, isFutureReplica = false, offsetCheckpoints)
assertTrue("Expected become leader transition to succeed",
partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints))
partition.makeLeader(
controllerId,
new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true),
0,
offsetCheckpoints))
assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds)
assertEquals(0L, partition.localLogOrException.highWatermark)
@ -1375,8 +1510,14 @@ class PartitionTest {
val controllerId = 0
val controllerEpoch = 3
val replicas = List[Integer](brokerId, brokerId + 1).asJava
val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
6, replicas, 1, replicas, false)
val leaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(6)
.setIsr(replicas)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(false)
partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints)
assertEquals(4, partition.localLogOrException.highWatermark)
}

View File

@ -23,6 +23,9 @@ import kafka.cluster.{Broker, EndPoint}
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.{LeaderAndIsrResponseData, StopReplicaResponseData}
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError
import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{AbstractControlRequest, AbstractResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, StopReplicaRequest, StopReplicaResponse, UpdateMetadataRequest}
@ -70,18 +73,21 @@ class ControllerChannelManagerTest {
val leaderAndIsrRequest = leaderAndIsrRequests.head
assertEquals(controllerId, leaderAndIsrRequest.controllerId)
assertEquals(controllerEpoch, leaderAndIsrRequest.controllerEpoch)
assertEquals(partitions.keySet, leaderAndIsrRequest.partitionStates.keySet.asScala)
assertEquals(partitions.keySet,
leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex)).toSet)
assertEquals(partitions.map { case (k, v) => (k, v.leader) },
leaderAndIsrRequest.partitionStates.asScala.mapValues(_.basePartitionState.leader).toMap)
leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.leader).toMap)
assertEquals(partitions.map { case (k, v) => (k, v.isr) },
leaderAndIsrRequest.partitionStates.asScala.mapValues(_.basePartitionState.isr.asScala).toMap)
leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.isr.asScala).toMap)
applyLeaderAndIsrResponseCallbacks(Errors.NONE, batch.sentRequests(2).toList)
assertEquals(1, batch.sentEvents.size)
val LeaderAndIsrResponseReceived(response, brokerId) = batch.sentEvents.head
val leaderAndIsrResponse = response.asInstanceOf[LeaderAndIsrResponse]
assertEquals(2, brokerId)
assertEquals(partitions.keySet, response.asInstanceOf[LeaderAndIsrResponse].responses.keySet.asScala)
assertEquals(partitions.keySet,
leaderAndIsrResponse.partitions.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex)).toSet)
}
@Test
@ -106,8 +112,10 @@ class ControllerChannelManagerTest {
assertEquals(1, updateMetadataRequests.size)
val leaderAndIsrRequest = leaderAndIsrRequests.head
assertEquals(Set(partition), leaderAndIsrRequest.partitionStates.keySet.asScala)
assertTrue(leaderAndIsrRequest.partitionStates.get(partition).isNew)
val partitionStates = leaderAndIsrRequest.partitionStates.asScala
assertEquals(Seq(partition), partitionStates.map(p => new TopicPartition(p.topicName, p.partitionIndex)))
val partitionState = partitionStates.find(p => p.topicName == partition.topic && p.partitionIndex == partition.partition)
assertEquals(Some(true), partitionState.map(_.isNew))
}
@Test
@ -139,7 +147,7 @@ class ControllerChannelManagerTest {
assertEquals(1, leaderAndIsrRequests.size)
assertEquals(1, updateMetadataRequests.size)
val leaderAndIsrRequest = leaderAndIsrRequests.head
assertEquals(Set(partition), leaderAndIsrRequest.partitionStates.keySet.asScala)
assertEquals(Seq(partition), leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex)))
}
}
@ -203,11 +211,12 @@ class ControllerChannelManagerTest {
assertEquals(1, updateMetadataRequests.size)
val updateMetadataRequest = updateMetadataRequests.head
assertEquals(3, updateMetadataRequest.partitionStates.size)
val partitionStates = updateMetadataRequest.partitionStates.asScala.toBuffer
assertEquals(3, partitionStates.size)
assertEquals(partitions.map { case (k, v) => (k, v.leader) },
updateMetadataRequest.partitionStates.asScala.map { case (k, v) => (k, v.basePartitionState.leader) })
partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader)).toMap)
assertEquals(partitions.map { case (k, v) => (k, v.isr) },
updateMetadataRequest.partitionStates.asScala.map { case (k, v) => (k, v.basePartitionState.isr.asScala) })
partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr.asScala)).toMap)
assertEquals(controllerId, updateMetadataRequest.controllerId)
assertEquals(controllerEpoch, updateMetadataRequest.controllerEpoch)
@ -238,7 +247,7 @@ class ControllerChannelManagerTest {
assertEquals(1, updateMetadataRequests.size)
val updateMetadataRequest = updateMetadataRequests.head
assertEquals(0, updateMetadataRequest.partitionStates.size)
assertEquals(0, updateMetadataRequest.partitionStates.asScala.size)
assertEquals(3, updateMetadataRequest.liveBrokers.size)
assertEquals(Set(1, 2, 3), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet)
}
@ -268,19 +277,18 @@ class ControllerChannelManagerTest {
assertEquals(1, updateMetadataRequests.size)
val updateMetadataRequest = updateMetadataRequests.head
assertEquals(3, updateMetadataRequest.partitionStates.size)
assertEquals(3, updateMetadataRequest.partitionStates.asScala.size)
assertTrue(updateMetadataRequest.partitionStates.asScala
.filterKeys(_.topic == "foo")
.values
.map(_.basePartitionState.leader)
.filter(_.topicName == "foo")
.map(_.leader)
.forall(leaderId => leaderId == LeaderAndIsr.LeaderDuringDelete))
assertEquals(partitions.filter { case (k, _) => k.topic == "bar" }.map { case (k, v) => (k, v.leader) },
updateMetadataRequest.partitionStates.asScala.filter { case (k, _) => k.topic == "bar" }.map { case (k, v) =>
(k, v.basePartitionState.leader) })
updateMetadataRequest.partitionStates.asScala.filter(ps => ps.topicName == "bar").map { ps =>
(new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader) }.toMap)
assertEquals(partitions.map { case (k, v) => (k, v.isr) },
updateMetadataRequest.partitionStates.asScala.map { case (k, v) => (k, v.basePartitionState.isr.asScala) })
updateMetadataRequest.partitionStates.asScala.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr.asScala)).toMap)
assertEquals(3, updateMetadataRequest.liveBrokers.size)
assertEquals(Set(1, 2, 3), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet)
@ -306,7 +314,7 @@ class ControllerChannelManagerTest {
assertEquals(1, updateMetadataRequests.size)
val updateMetadataRequest = updateMetadataRequests.head
assertEquals(0, updateMetadataRequest.partitionStates.size)
assertEquals(0, updateMetadataRequest.partitionStates.asScala.size)
assertEquals(2, updateMetadataRequest.liveBrokers.size)
assertEquals(Set(1, 2), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet)
}
@ -415,7 +423,7 @@ class ControllerChannelManagerTest {
val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(2)
assertEquals(1, sentStopReplicaRequests.size)
assertEquals(partitions, sentStopReplicaRequests.flatMap(_.partitions.asScala).toSet)
assertTrue(sentStopReplicaRequests.forall(_.deletePartitions()))
assertTrue(sentStopReplicaRequests.forall(_.deletePartitions))
// No events will be sent after the response returns
applyStopReplicaResponseCallbacks(Errors.NONE, batch.sentRequests(2).toList)
@ -620,8 +628,18 @@ class ControllerChannelManagerTest {
private def applyStopReplicaResponseCallbacks(error: Errors, sentRequests: List[SentRequest]): Unit = {
sentRequests.filter(_.responseCallback != null).foreach { sentRequest =>
val stopReplicaRequest = sentRequest.request.build().asInstanceOf[StopReplicaRequest]
val partitionErrorMap = stopReplicaRequest.partitions.asScala.map(_ -> error).toMap.asJava
val stopReplicaResponse = new StopReplicaResponse(error, partitionErrorMap)
val stopReplicaResponse =
if (error == Errors.NONE) {
val partitionErrors = stopReplicaRequest.partitions.asScala.map { tp =>
new StopReplicaPartitionError()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setErrorCode(error.code)
}.toBuffer.asJava
new StopReplicaResponse(new StopReplicaResponseData().setPartitionErrors(partitionErrors))
} else {
stopReplicaRequest.getErrorResponse(error.exception)
}
sentRequest.responseCallback.apply(stopReplicaResponse)
}
}
@ -629,9 +647,16 @@ class ControllerChannelManagerTest {
private def applyLeaderAndIsrResponseCallbacks(error: Errors, sentRequests: List[SentRequest]): Unit = {
sentRequests.filter(_.request.apiKey == ApiKeys.LEADER_AND_ISR).filter(_.responseCallback != null).foreach { sentRequest =>
val leaderAndIsrRequest = sentRequest.request.build().asInstanceOf[LeaderAndIsrRequest]
val partitionErrorMap = leaderAndIsrRequest.partitionStates.asScala.keySet.map(_ -> error).toMap.asJava
val leaderAndIsrResponse = new LeaderAndIsrResponse(error, partitionErrorMap)
sentRequest.responseCallback.apply(leaderAndIsrResponse)
val partitionErrors = leaderAndIsrRequest.partitionStates.asScala.map(p =>
new LeaderAndIsrPartitionError()
.setTopicName(p.topicName)
.setPartitionIndex(p.partitionIndex)
.setErrorCode(error.code))
val leaderAndIsrResponse = new LeaderAndIsrResponse(
new LeaderAndIsrResponseData()
.setErrorCode(error.code)
.setPartitionErrors(partitionErrors.toBuffer.asJava))
sentRequest.responseCallback(leaderAndIsrResponse)
}
}

View File

@ -143,10 +143,10 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
val offlineReplicaPartitionInfo = server.metadataCache.getPartitionInfo(topic, 0).get
assertEquals(1, offlineReplicaPartitionInfo.offlineReplicas.size())
assertEquals(testBroker.config.brokerId, offlineReplicaPartitionInfo.offlineReplicas.get(0))
assertEquals(assignment(0).asJava, offlineReplicaPartitionInfo.basePartitionState.replicas)
assertEquals(Seq(remainingBrokers.head.config.brokerId).asJava, offlineReplicaPartitionInfo.basePartitionState.isr)
assertEquals(assignment(0).asJava, offlineReplicaPartitionInfo.replicas)
assertEquals(Seq(remainingBrokers.head.config.brokerId).asJava, offlineReplicaPartitionInfo.isr)
val onlinePartitionInfo = server.metadataCache.getPartitionInfo(topic, 1).get
assertEquals(assignment(1).asJava, onlinePartitionInfo.basePartitionState.replicas)
assertEquals(assignment(1).asJava, onlinePartitionInfo.replicas)
assertTrue(onlinePartitionInfo.offlineReplicas.isEmpty)
}
@ -158,7 +158,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
val partitionInfoOpt = server.metadataCache.getPartitionInfo(topic, partitionId)
if (partitionInfoOpt.isDefined) {
val partitionInfo = partitionInfoOpt.get
!partitionInfo.offlineReplicas.isEmpty || !partitionInfo.basePartitionState.replicas.asScala.equals(replicas)
!partitionInfo.offlineReplicas.isEmpty || !partitionInfo.replicas.asScala.equals(replicas)
} else {
true
}
@ -204,8 +204,8 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
val partitionInfoOpt = broker.metadataCache.getPartitionInfo(topic, 0)
if (partitionInfoOpt.isDefined) {
val partitionInfo = partitionInfoOpt.get
(!partitionInfo.offlineReplicas.isEmpty && partitionInfo.basePartitionState.leader == -1
&& !partitionInfo.basePartitionState.replicas.isEmpty && !partitionInfo.basePartitionState.isr.isEmpty)
(!partitionInfo.offlineReplicas.isEmpty && partitionInfo.leader == -1
&& !partitionInfo.replicas.isEmpty && !partitionInfo.isr.isEmpty)
} else {
false
}
@ -463,14 +463,14 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
var activeServers = servers.filter(s => s.config.brokerId != 2)
// wait for the update metadata request to trickle to the brokers
TestUtils.waitUntilTrue(() =>
activeServers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.basePartitionState.isr.size != 3),
activeServers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.isr.size != 3),
"Topic test not created after timeout")
assertEquals(0, partitionsRemaining.size)
var partitionStateInfo = activeServers.head.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get
var leaderAfterShutdown = partitionStateInfo.basePartitionState.leader
var leaderAfterShutdown = partitionStateInfo.leader
assertEquals(0, leaderAfterShutdown)
assertEquals(2, partitionStateInfo.basePartitionState.isr.size)
assertEquals(List(0,1), partitionStateInfo.basePartitionState.isr.asScala)
assertEquals(2, partitionStateInfo.isr.size)
assertEquals(List(0,1), partitionStateInfo.isr.asScala)
controller.controlledShutdown(1, servers.find(_.config.brokerId == 1).get.kafkaController.brokerEpoch, controlledShutdownCallback)
partitionsRemaining = resultQueue.take() match {
case Success(partitions) => partitions
@ -479,15 +479,15 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
assertEquals(0, partitionsRemaining.size)
activeServers = servers.filter(s => s.config.brokerId == 0)
partitionStateInfo = activeServers.head.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get
leaderAfterShutdown = partitionStateInfo.basePartitionState.leader
leaderAfterShutdown = partitionStateInfo.leader
assertEquals(0, leaderAfterShutdown)
assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.basePartitionState.leader == 0))
assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.leader == 0))
controller.controlledShutdown(0, servers.find(_.config.brokerId == 0).get.kafkaController.brokerEpoch, controlledShutdownCallback)
partitionsRemaining = resultQueue.take().get
assertEquals(1, partitionsRemaining.size)
// leader doesn't change since all the replicas are shut down
assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.basePartitionState.leader == 0))
assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.leader == 0))
}
@Test

View File

@ -255,7 +255,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
//make sure follower server joins the ISR
TestUtils.waitUntilTrue(() => {
val partitionInfoOpt = followerServer.metadataCache.getPartitionInfo(topic, partitionId)
partitionInfoOpt.isDefined && partitionInfoOpt.get.basePartitionState.isr.contains(followerId)
partitionInfoOpt.isDefined && partitionInfoOpt.get.isr.contains(followerId)
}, "Inconsistent metadata after first server startup")
servers.filter(server => server.config.brokerId == leaderId).foreach(server => shutdownServer(server))

View File

@ -32,7 +32,7 @@ import kafka.security.CredentialProvider
import kafka.server.{KafkaConfig, ThrottledChannel}
import kafka.utils.Implicits._
import kafka.utils.{CoreUtils, TestUtils}
import org.apache.kafka.common.{Endpoint, TopicPartition}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.KafkaChannel.ChannelMuteState

View File

@ -24,10 +24,11 @@ import kafka.utils.TestUtils
import kafka.utils.TestUtils.createTopic
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.UpdateMetadataRequest.EndPoint
import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Time
@ -132,10 +133,17 @@ class BrokerEpochIntegrationTest extends ZooKeeperTestHarness {
try {
// Send LeaderAndIsr request with correct broker epoch
{
val partitionStates = Map(
tp -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId2, LeaderAndIsr.initialLeaderEpoch + 1,
Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion,
Seq(0, 1).map(Integer.valueOf).asJava, false)
val partitionStates = Seq(
new LeaderAndIsrPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId2)
.setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch + 1)
.setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava)
.setZkVersion(LeaderAndIsr.initialZKVersion)
.setReplicas(Seq(0, 1).map(Integer.valueOf).asJava)
.setIsNew(false)
)
val requestBuilder = new LeaderAndIsrRequest.Builder(
ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, controllerEpoch,
@ -153,32 +161,43 @@ class BrokerEpochIntegrationTest extends ZooKeeperTestHarness {
// Send UpdateMetadata request with correct broker epoch
{
val partitionStates = Map(
tp -> new UpdateMetadataRequest.PartitionState(controllerEpoch, brokerId2, LeaderAndIsr.initialLeaderEpoch + 1,
Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion,
Seq(0, 1).map(Integer.valueOf).asJava, Seq.empty.asJava)
)
val liverBrokers = brokerAndEpochs.map { brokerAndEpoch =>
val broker = brokerAndEpoch._1
val partitionStates = Seq(
new UpdateMetadataPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId2)
.setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch + 1)
.setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava)
.setZkVersion(LeaderAndIsr.initialZKVersion)
.setReplicas(Seq(0, 1).map(Integer.valueOf).asJava))
val liveBrokers = brokerAndEpochs.map { case (broker, _) =>
val securityProtocol = SecurityProtocol.PLAINTEXT
val listenerName = ListenerName.forSecurityProtocol(securityProtocol)
val node = broker.node(listenerName)
val endPoints = Seq(new EndPoint(node.host, node.port, securityProtocol, listenerName))
new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull)
}
val endpoints = Seq(new UpdateMetadataEndpoint()
.setHost(node.host)
.setPort(node.port)
.setSecurityProtocol(securityProtocol.id)
.setListener(listenerName.value))
new UpdateMetadataBroker()
.setId(broker.id)
.setEndpoints(endpoints.asJava)
.setRack(broker.rack.orNull)
}.toBuffer
val requestBuilder = new UpdateMetadataRequest.Builder(
ApiKeys.UPDATE_METADATA.latestVersion, controllerId, controllerEpoch,
epochInRequest,
partitionStates.asJava, liverBrokers.toSet.asJava)
partitionStates.asJava, liveBrokers.asJava)
if (isEpochInRequestStale) {
sendAndVerifyStaleBrokerEpochInResponse(controllerChannelManager, requestBuilder)
}
else {
sendAndVerifySuccessfulResponse(controllerChannelManager, requestBuilder)
TestUtils.waitUntilMetadataIsPropagated(Seq(broker2), tp.topic(), tp.partition(), 10000)
TestUtils.waitUntilMetadataIsPropagated(Seq(broker2), tp.topic, tp.partition, 10000)
assertEquals(brokerId2,
broker2.metadataCache.getPartitionInfo(tp.topic(), tp.partition()).get.basePartitionState.leader)
broker2.metadataCache.getPartitionInfo(tp.topic, tp.partition).get.leader)
}
}

View File

@ -34,6 +34,7 @@ import org.junit.Assert._
import org.junit.Test
import scala.collection.JavaConverters._
import scala.collection.Seq
import scala.util.Random
/**
@ -556,7 +557,7 @@ class FetchRequestTest extends BaseRequestTest {
}
private def records(partitionData: FetchResponse.PartitionData[MemoryRecords]): Seq[Record] = {
partitionData.records.records.asScala.toIndexedSeq
partitionData.records.records.asScala.toBuffer
}
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse[MemoryRecords],
@ -574,7 +575,7 @@ class FetchRequestTest extends BaseRequestTest {
val records = partitionData.records
responseBufferSize += records.sizeInBytes
val batches = records.batches.asScala.toIndexedSeq
val batches = records.batches.asScala.toBuffer
assertTrue(batches.size < numMessagesPerPartition)
val batchesSize = batches.map(_.sizeInBytes).sum
responseSize += batchesSize

View File

@ -41,7 +41,6 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint}
import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry
import org.apache.kafka.common.requests.{FetchMetadata => JFetchMetadata, _}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
@ -50,6 +49,7 @@ import EasyMock._
import org.apache.kafka.common.message.{HeartbeatRequestData, JoinGroupRequestData, OffsetCommitRequestData, OffsetCommitResponseData, SyncGroupRequestData, TxnOffsetCommitRequestData}
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.replica.ClientMetadata
import org.apache.kafka.server.authorizer.Authorizer
import org.junit.Assert.{assertEquals, assertNull, assertTrue}
@ -714,14 +714,34 @@ class KafkaApisTest {
private def updateMetadataCacheWithInconsistentListeners(): (ListenerName, ListenerName) = {
val plaintextListener = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
val anotherListener = new ListenerName("LISTENER2")
val brokers = Set(
new Broker(0, Seq(new EndPoint("broker0", 9092, SecurityProtocol.PLAINTEXT, plaintextListener),
new EndPoint("broker0", 9093, SecurityProtocol.PLAINTEXT, anotherListener)).asJava, "rack"),
new Broker(1, Seq(new EndPoint("broker1", 9092, SecurityProtocol.PLAINTEXT, plaintextListener)).asJava,
"rack")
val brokers = Seq(
new UpdateMetadataBroker()
.setId(0)
.setRack("rack")
.setEndpoints(Seq(
new UpdateMetadataEndpoint()
.setHost("broker0")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListener.value),
new UpdateMetadataEndpoint()
.setHost("broker0")
.setPort(9093)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(anotherListener.value)
).asJava),
new UpdateMetadataBroker()
.setId(1)
.setRack("rack")
.setEndpoints(Seq(
new UpdateMetadataEndpoint()
.setHost("broker1")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListener.value)).asJava)
)
val updateMetadataRequest = new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0,
0, 0, Map.empty[TopicPartition, UpdateMetadataRequest.PartitionState].asJava, brokers.asJava).build()
0, 0, Seq.empty[UpdateMetadataPartitionState].asJava, brokers.asJava).build()
metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest)
(plaintextListener, anotherListener)
}
@ -815,12 +835,29 @@ class KafkaApisTest {
private def setupBasicMetadataCache(topic: String, numPartitions: Int): Unit = {
val replicas = List(0.asInstanceOf[Integer]).asJava
val partitionState = new UpdateMetadataRequest.PartitionState(1, 0, 1, replicas, 0, replicas, Collections.emptyList())
def createPartitionState(partition: Int) = new UpdateMetadataPartitionState()
.setTopicName(topic)
.setPartitionIndex(partition)
.setControllerEpoch(1)
.setLeader(0)
.setLeaderEpoch(1)
.setReplicas(replicas)
.setZkVersion(0)
.setReplicas(replicas)
val plaintextListener = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
val broker = new Broker(0, Seq(new EndPoint("broker0", 9092, SecurityProtocol.PLAINTEXT, plaintextListener)).asJava, "rack")
val partitions = (0 until numPartitions).map(new TopicPartition(topic, _) -> partitionState).toMap
val broker = new UpdateMetadataBroker()
.setId(0)
.setRack("rack")
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("broker0")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListener.value)).asJava)
val partitionStates = (0 until numPartitions).map(createPartitionState)
val updateMetadataRequest = new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0,
0, 0, partitions.asJava, Set(broker).asJava).build()
0, 0, partitionStates.asJava, Seq(broker).asJava).build()
metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest)
}
}

View File

@ -28,6 +28,7 @@ import kafka.cluster.Broker
import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger}
import kafka.utils.TestUtils._
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
@ -93,12 +94,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
servers.head.startup()
//make sure second server joins the ISR
TestUtils.waitUntilTrue(() => {
val partitionInfoOpt = servers.last.metadataCache.getPartitionInfo(topic, partitionId)
if (partitionInfoOpt.isDefined) {
partitionInfoOpt.get.basePartitionState.isr.size() == 2
} else {
false
}
servers.last.metadataCache.getPartitionInfo(topic, partitionId).exists(_.isr.size == 2)
}, "Inconsistent metadata after second broker startup")
servers.last.shutdown()
@ -145,13 +141,21 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
controllerChannelManager.startup()
try {
val staleControllerEpoch = 0
val partitionStates = Map(
new TopicPartition(topic, partitionId) -> new LeaderAndIsrRequest.PartitionState(2, brokerId2, LeaderAndIsr.initialLeaderEpoch,
Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion,
Seq(0, 1).map(Integer.valueOf).asJava, false)
val partitionStates = Seq(
new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(partitionId)
.setControllerEpoch(2)
.setLeader(brokerId2)
.setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch)
.setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava)
.setZkVersion(LeaderAndIsr.initialZKVersion)
.setReplicas(Seq(0, 1).map(Integer.valueOf).asJava)
.setIsNew(false)
)
val requestBuilder = new LeaderAndIsrRequest.Builder(
ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, staleControllerEpoch, servers(brokerId2).kafkaController.brokerEpoch ,partitionStates.asJava, nodes.toSet.asJava)
ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, staleControllerEpoch,
servers(brokerId2).kafkaController.brokerEpoch, partitionStates.asJava, nodes.toSet.asJava)
controllerChannelManager.sendRequest(brokerId2, requestBuilder, staleControllerEpochCallback)
TestUtils.waitUntilTrue(() => staleControllerEpochDetected, "Controller epoch should be stale")

View File

@ -20,14 +20,14 @@ import java.util
import java.util.Optional
import util.Arrays.asList
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.UpdateMetadataRequest
import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.junit.Test
import org.junit.Assert._
import org.scalatest.Assertions
import scala.collection.JavaConverters._
@ -53,22 +53,57 @@ class MetadataCacheTest {
val controllerId = 2
val controllerEpoch = 1
def endPoints(brokerId: Int): Seq[EndPoint] = {
def endpoints(brokerId: Int): Seq[UpdateMetadataEndpoint] = {
val host = s"foo-$brokerId"
Seq(
new EndPoint(host, 9092, SecurityProtocol.PLAINTEXT, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)),
new EndPoint(host, 9093, SecurityProtocol.SSL, ListenerName.forSecurityProtocol(SecurityProtocol.SSL))
new UpdateMetadataEndpoint()
.setHost(host)
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT).value),
new UpdateMetadataEndpoint()
.setHost(host)
.setPort(9093)
.setSecurityProtocol(SecurityProtocol.SSL.id)
.setListener(ListenerName.forSecurityProtocol(SecurityProtocol.SSL).value)
)
}
val brokers = (0 to 4).map { brokerId =>
new Broker(brokerId, endPoints(brokerId).asJava, "rack1")
}.toSet
new UpdateMetadataBroker()
.setId(brokerId)
.setEndpoints(endpoints(brokerId).asJava)
.setRack("rack1")
}
val partitionStates = Map(
new TopicPartition(topic0, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 0, 0, asList(0, 1, 3), zkVersion, asList(0, 1, 3), asList()),
new TopicPartition(topic0, 1) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4), asList()),
new TopicPartition(topic1, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3), asList()))
val partitionStates = Seq(
new UpdateMetadataPartitionState()
.setTopicName(topic0)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(asList(0, 1, 3))
.setZkVersion(zkVersion)
.setReplicas(asList(0, 1, 3)),
new UpdateMetadataPartitionState()
.setTopicName(topic0)
.setPartitionIndex(1)
.setControllerEpoch(controllerEpoch)
.setLeader(1)
.setLeaderEpoch(1)
.setIsr(asList(1, 0))
.setZkVersion(zkVersion)
.setReplicas(asList(1, 2, 0, 4)),
new UpdateMetadataPartitionState()
.setTopicName(topic1)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(2)
.setLeaderEpoch(2)
.setIsr(asList(2, 1))
.setZkVersion(zkVersion)
.setReplicas(asList(2, 1, 3)))
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
@ -86,7 +121,7 @@ class MetadataCacheTest {
assertEquals(Errors.NONE, topicMetadata.error)
assertEquals(topic, topicMetadata.topic)
val topicPartitionStates = partitionStates.filter { case (tp, _) => tp.topic == topic }
val topicPartitionStates = partitionStates.filter { ps => ps.topicName == topic }
val partitionMetadatas = topicMetadata.partitionMetadata.asScala.sortBy(_.partition)
assertEquals(s"Unexpected partition count for topic $topic", topicPartitionStates.size, partitionMetadatas.size)
@ -94,14 +129,15 @@ class MetadataCacheTest {
assertEquals(Errors.NONE, partitionMetadata.error)
assertEquals(partitionId, partitionMetadata.partition)
val leader = partitionMetadata.leader
val partitionState = topicPartitionStates(new TopicPartition(topic, partitionId))
assertEquals(partitionState.basePartitionState.leader, leader.id)
assertEquals(Optional.of(partitionState.basePartitionState.leaderEpoch), partitionMetadata.leaderEpoch)
assertEquals(partitionState.basePartitionState.isr, partitionMetadata.isr.asScala.map(_.id).asJava)
assertEquals(partitionState.basePartitionState.replicas, partitionMetadata.replicas.asScala.map(_.id).asJava)
val endPoint = endPoints(partitionMetadata.leader.id).find(_.listenerName == listenerName).get
assertEquals(endPoint.host, leader.host)
assertEquals(endPoint.port, leader.port)
val partitionState = topicPartitionStates.find(_.partitionIndex == partitionId).getOrElse(
Assertions.fail(s"Unable to find partition state for partition $partitionId"))
assertEquals(partitionState.leader, leader.id)
assertEquals(Optional.of(partitionState.leaderEpoch), partitionMetadata.leaderEpoch)
assertEquals(partitionState.isr, partitionMetadata.isr.asScala.map(_.id).asJava)
assertEquals(partitionState.replicas, partitionMetadata.replicas.asScala.map(_.id).asJava)
val endpoint = endpoints(partitionMetadata.leader.id).find(_.listener == listenerName.value).get
assertEquals(endpoint.host, leader.host)
assertEquals(endpoint.port, leader.port)
}
}
@ -115,7 +151,13 @@ class MetadataCacheTest {
def getTopicMetadataPartitionLeaderNotAvailable(): Unit = {
val securityProtocol = SecurityProtocol.PLAINTEXT
val listenerName = ListenerName.forSecurityProtocol(securityProtocol)
val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, null))
val brokers = Seq(new UpdateMetadataBroker()
.setId(0)
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("foo")
.setPort(9092)
.setSecurityProtocol(securityProtocol.id)
.setListener(listenerName.value)).asJava))
verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, listenerName,
leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false)
verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, listenerName,
@ -127,10 +169,28 @@ class MetadataCacheTest {
val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
val sslListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.SSL)
val broker0Endpoints = Seq(
new EndPoint("host0", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName),
new EndPoint("host0", 9093, SecurityProtocol.SSL, sslListenerName))
val broker1Endpoints = Seq(new EndPoint("host1", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName))
val brokers = Set(new Broker(0, broker0Endpoints.asJava, null), new Broker(1, broker1Endpoints.asJava, null))
new UpdateMetadataEndpoint()
.setHost("host0")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListenerName.value),
new UpdateMetadataEndpoint()
.setHost("host0")
.setPort(9093)
.setSecurityProtocol(SecurityProtocol.SSL.id)
.setListener(sslListenerName.value))
val broker1Endpoints = Seq(new UpdateMetadataEndpoint()
.setHost("host1")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListenerName.value))
val brokers = Seq(
new UpdateMetadataBroker()
.setId(0)
.setEndpoints(broker0Endpoints.asJava),
new UpdateMetadataBroker()
.setId(1)
.setEndpoints(broker1Endpoints.asJava))
verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, sslListenerName,
leader = 1, Errors.LISTENER_NOT_FOUND, errorUnavailableListeners = true)
}
@ -140,15 +200,33 @@ class MetadataCacheTest {
val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
val sslListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.SSL)
val broker0Endpoints = Seq(
new EndPoint("host0", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName),
new EndPoint("host0", 9093, SecurityProtocol.SSL, sslListenerName))
val broker1Endpoints = Seq(new EndPoint("host1", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName))
val brokers = Set(new Broker(0, broker0Endpoints.asJava, null), new Broker(1, broker1Endpoints.asJava, null))
new UpdateMetadataEndpoint()
.setHost("host0")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListenerName.value),
new UpdateMetadataEndpoint()
.setHost("host0")
.setPort(9093)
.setSecurityProtocol(SecurityProtocol.SSL.id)
.setListener(sslListenerName.value))
val broker1Endpoints = Seq(new UpdateMetadataEndpoint()
.setHost("host1")
.setPort(9092)
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
.setListener(plaintextListenerName.value))
val brokers = Seq(
new UpdateMetadataBroker()
.setId(0)
.setEndpoints(broker0Endpoints.asJava),
new UpdateMetadataBroker()
.setId(1)
.setEndpoints(broker1Endpoints.asJava))
verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, sslListenerName,
leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false)
}
private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers: Set[Broker],
private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers: Seq[UpdateMetadataBroker],
listenerName: ListenerName,
leader: Int,
expectedError: Errors,
@ -162,8 +240,15 @@ class MetadataCacheTest {
val controllerEpoch = 1
val leaderEpoch = 1
val partitionStates = Map(
new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0), asList()))
val partitionStates = Seq(new UpdateMetadataPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(asList(0))
.setZkVersion(zkVersion)
.setReplicas(asList(0)))
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
@ -198,7 +283,13 @@ class MetadataCacheTest {
val controllerEpoch = 1
val securityProtocol = SecurityProtocol.PLAINTEXT
val listenerName = ListenerName.forSecurityProtocol(securityProtocol)
val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, null))
val brokers = Seq(new UpdateMetadataBroker()
.setId(0)
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("foo")
.setPort(9092)
.setSecurityProtocol(securityProtocol.id)
.setListener(listenerName.value)).asJava))
// replica 1 is not available
val leader = 0
@ -206,8 +297,16 @@ class MetadataCacheTest {
val replicas = asList[Integer](0, 1)
val isr = asList[Integer](0)
val partitionStates = Map(
new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList()))
val partitionStates = Seq(
new UpdateMetadataPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(zkVersion)
.setReplicas(replicas))
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
@ -258,7 +357,14 @@ class MetadataCacheTest {
val controllerEpoch = 1
val securityProtocol = SecurityProtocol.PLAINTEXT
val listenerName = ListenerName.forSecurityProtocol(securityProtocol)
val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, "rack1"))
val brokers = Seq(new UpdateMetadataBroker()
.setId(0)
.setRack("rack1")
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("foo")
.setPort(9092)
.setSecurityProtocol(securityProtocol.id)
.setListener(listenerName.value)).asJava))
// replica 1 is not available
val leader = 0
@ -266,8 +372,15 @@ class MetadataCacheTest {
val replicas = asList[Integer](0)
val isr = asList[Integer](0, 1)
val partitionStates = Map(
new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList()))
val partitionStates = Seq(new UpdateMetadataPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(zkVersion)
.setReplicas(replicas))
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
@ -312,15 +425,28 @@ class MetadataCacheTest {
val topic = "topic"
val cache = new MetadataCache(1)
val securityProtocol = SecurityProtocol.PLAINTEXT
val brokers = Set(new Broker(0,
Seq(new EndPoint("foo", 9092, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, ""))
val brokers = Seq(new UpdateMetadataBroker()
.setId(0)
.setRack("")
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("foo")
.setPort(9092)
.setSecurityProtocol(securityProtocol.id)
.setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava))
val controllerEpoch = 1
val leader = 0
val leaderEpoch = 0
val replicas = asList[Integer](0)
val isr = asList[Integer](0, 1)
val partitionStates = Map(
new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList()))
val partitionStates = Seq(new UpdateMetadataPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(3)
.setReplicas(replicas))
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava,
brokers.asJava).build()
@ -337,31 +463,44 @@ class MetadataCacheTest {
val topic = "topic"
val cache = new MetadataCache(1)
def updateCache(brokerIds: Set[Int]): Unit = {
def updateCache(brokerIds: Seq[Int]): Unit = {
val brokers = brokerIds.map { brokerId =>
val securityProtocol = SecurityProtocol.PLAINTEXT
new Broker(brokerId, Seq(
new EndPoint("foo", 9092, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, "")
new UpdateMetadataBroker()
.setId(brokerId)
.setRack("")
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("foo")
.setPort(9092)
.setSecurityProtocol(securityProtocol.id)
.setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)
}
val controllerEpoch = 1
val leader = 0
val leaderEpoch = 0
val replicas = asList[Integer](0)
val isr = asList[Integer](0, 1)
val partitionStates = Map(
new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList()))
val partitionStates = Seq(new UpdateMetadataPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setZkVersion(3)
.setReplicas(replicas))
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava,
brokers.asJava).build()
cache.updateMetadata(15, updateMetadataRequest)
}
val initialBrokerIds = (0 to 2).toSet
val initialBrokerIds = (0 to 2)
updateCache(initialBrokerIds)
val aliveBrokersFromCache = cache.getAliveBrokers
// This should not change `aliveBrokersFromCache`
updateCache((0 to 3).toSet)
assertEquals(initialBrokerIds, aliveBrokersFromCache.map(_.id).toSet)
updateCache((0 to 3))
assertEquals(initialBrokerIds.toSet, aliveBrokersFromCache.map(_.id).toSet)
}
}

View File

@ -36,6 +36,7 @@ import kafka.utils.TestUtils.createBroker
import kafka.utils.timer.MockTimer
import kafka.utils.{MockScheduler, MockTime, TestUtils}
import kafka.zk.KafkaZkClient
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record._
@ -170,8 +171,16 @@ class ReplicaManagerTest {
new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints))
// Make this replica the leader.
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(false)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
rm.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
@ -184,8 +193,16 @@ class ReplicaManagerTest {
// Make this replica the follower
val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(1)
.setLeaderEpoch(1)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(false)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ())
@ -209,8 +226,16 @@ class ReplicaManagerTest {
// Make this replica the leader.
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(true)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
@ -261,8 +286,16 @@ class ReplicaManagerTest {
// Make this replica the leader.
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(true)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
@ -358,8 +391,16 @@ class ReplicaManagerTest {
// Make this replica the leader.
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(true)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
@ -425,8 +466,16 @@ class ReplicaManagerTest {
// Make this replica the leader.
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(false)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava).build()
rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
rm.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
@ -469,10 +518,18 @@ class ReplicaManagerTest {
val replicas = aliveBrokersIds.toList.map(Int.box).asJava
// Broker 0 becomes leader of the partition
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState(0, 0, leaderEpoch,
replicas, 0, replicas, true)
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(leaderEpoch)
.setIsr(replicas)
.setZkVersion(0)
.setReplicas(replicas)
.setIsNew(true)
val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
Map(tp -> leaderAndIsrPartitionState).asJava,
Seq(leaderAndIsrPartitionState).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
val leaderAndIsrResponse = replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ())
assertEquals(Errors.NONE, leaderAndIsrResponse.error)
@ -565,9 +622,27 @@ class ReplicaManagerTest {
val partition0Replicas = Seq[Integer](0, 1).asJava
val partition1Replicas = Seq[Integer](0, 2).asJava
val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(
tp0 -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, partition0Replicas, 0, partition0Replicas, true),
tp1 -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, partition1Replicas, 0, partition1Replicas, true)
Seq(
new LeaderAndIsrPartitionState()
.setTopicName(tp0.topic)
.setPartitionIndex(tp0.partition)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(partition0Replicas)
.setZkVersion(0)
.setReplicas(partition0Replicas)
.setIsNew(true),
new LeaderAndIsrPartitionState()
.setTopicName(tp1.topic)
.setPartitionIndex(tp1.partition)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(partition1Replicas)
.setZkVersion(0)
.setReplicas(partition1Replicas)
.setIsNew(true)
).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ())
@ -653,11 +728,12 @@ class ReplicaManagerTest {
topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = true)
// Initialize partition state to follower, with leader = 1, leaderEpoch = 1
val partition = replicaManager.createPartition(new TopicPartition(topic, topicPartition))
val tp = new TopicPartition(topic, topicPartition)
val partition = replicaManager.createPartition(tp)
val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints)
partition.createLogIfNotExists(followerBrokerId, isNew = false, isFutureReplica = false, offsetCheckpoints)
partition.makeFollower(controllerId,
leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds),
leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds),
correlationId, offsetCheckpoints)
// Make local partition a follower - because epoch increased by more than 1, truncation should
@ -665,8 +741,7 @@ class ReplicaManagerTest {
leaderEpoch += leaderEpochIncrement
val leaderAndIsrRequest0 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion,
controllerId, controllerEpoch, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, topicPartition) ->
leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava,
Seq(leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava,
Set(new Node(followerBrokerId, "host1", 0),
new Node(leaderBrokerId, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest0,
@ -693,13 +768,14 @@ class ReplicaManagerTest {
topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId,
leaderBrokerId, countDownLatch, expectTruncation = true)
val partition = replicaManager.createPartition(new TopicPartition(topic, topicPartition))
val tp = new TopicPartition(topic, topicPartition)
val partition = replicaManager.createPartition(tp)
val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints)
partition.createLogIfNotExists(leaderBrokerId, isNew = false, isFutureReplica = false, offsetCheckpoints)
partition.makeLeader(
controllerId,
leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds),
leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds),
correlationId,
offsetCheckpoints
)
@ -737,8 +813,16 @@ class ReplicaManagerTest {
// Make this replica the follower
val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(1)
.setLeaderEpoch(1)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(false)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ())
@ -778,8 +862,16 @@ class ReplicaManagerTest {
// Make this replica the follower
val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch,
collection.immutable.Map(new TopicPartition(topic, 0) ->
new LeaderAndIsrRequest.PartitionState(0, 0, 1, brokerList, 0, brokerList, false)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(0)
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(1)
.setIsr(brokerList)
.setZkVersion(0)
.setReplicas(brokerList)
.setIsNew(false)).asJava,
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ())
@ -960,11 +1052,20 @@ class ReplicaManagerTest {
(replicaManager, mockLogMgr)
}
private def leaderAndIsrPartitionState(leaderEpoch: Int,
private def leaderAndIsrPartitionState(topicPartition: TopicPartition,
leaderEpoch: Int,
leaderBrokerId: Int,
aliveBrokerIds: Seq[Integer]) : LeaderAndIsrRequest.PartitionState = {
new LeaderAndIsrRequest.PartitionState(controllerEpoch, leaderBrokerId, leaderEpoch, aliveBrokerIds.asJava,
zkVersion, aliveBrokerIds.asJava, false)
aliveBrokerIds: Seq[Integer]): LeaderAndIsrPartitionState = {
new LeaderAndIsrPartitionState()
.setTopicName(topic)
.setPartitionIndex(topicPartition.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(leaderBrokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(aliveBrokerIds.asJava)
.setZkVersion(zkVersion)
.setReplicas(aliveBrokerIds.asJava)
.setIsNew(false)
}
private class CallbackResult[T] {
@ -1117,11 +1218,27 @@ class ReplicaManagerTest {
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion,
controllerId, 0, brokerEpoch,
collection.immutable.Map(
tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch,
partition0Replicas, 0, partition0Replicas, true),
tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 1, leaderEpoch,
partition1Replicas, 0, partition1Replicas, true)
Seq(
new LeaderAndIsrPartitionState()
.setTopicName(tp0.topic)
.setPartitionIndex(tp0.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(0)
.setLeaderEpoch(leaderEpoch)
.setIsr(partition0Replicas)
.setZkVersion(0)
.setReplicas(partition0Replicas)
.setIsNew(true),
new LeaderAndIsrPartitionState()
.setTopicName(tp1.topic)
.setPartitionIndex(tp1.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(1)
.setLeaderEpoch(leaderEpoch)
.setIsr(partition1Replicas)
.setZkVersion(0)
.setReplicas(partition1Replicas)
.setIsNew(true)
).asJava,
Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build()
@ -1131,11 +1248,27 @@ class ReplicaManagerTest {
// make broker 0 the leader of partition 1 so broker 1 loses its leadership position
val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId,
controllerEpoch, brokerEpoch,
collection.immutable.Map(
tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement,
partition0Replicas, 0, partition0Replicas, true),
tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement,
partition1Replicas, 0, partition1Replicas, true)
Seq(
new LeaderAndIsrPartitionState()
.setTopicName(tp0.topic)
.setPartitionIndex(tp0.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(0)
.setLeaderEpoch(leaderEpoch + leaderEpochIncrement)
.setIsr(partition0Replicas)
.setZkVersion(0)
.setReplicas(partition0Replicas)
.setIsNew(true),
new LeaderAndIsrPartitionState()
.setTopicName(tp1.topic)
.setPartitionIndex(tp1.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(0)
.setLeaderEpoch(leaderEpoch + leaderEpochIncrement)
.setIsr(partition1Replicas)
.setZkVersion(0)
.setReplicas(partition1Replicas)
.setIsNew(true)
).asJava,
Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build()
@ -1173,11 +1306,27 @@ class ReplicaManagerTest {
val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion,
controllerId, 0, brokerEpoch,
collection.immutable.Map(
tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 1, leaderEpoch,
partition0Replicas, 0, partition0Replicas, true),
tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 1, leaderEpoch,
partition1Replicas, 0, partition1Replicas, true)
Seq(
new LeaderAndIsrPartitionState()
.setTopicName(tp0.topic)
.setPartitionIndex(tp0.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(1)
.setLeaderEpoch(leaderEpoch)
.setIsr(partition0Replicas)
.setZkVersion(0)
.setReplicas(partition0Replicas)
.setIsNew(true),
new LeaderAndIsrPartitionState()
.setTopicName(tp1.topic)
.setPartitionIndex(tp1.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(1)
.setLeaderEpoch(leaderEpoch)
.setIsr(partition1Replicas)
.setZkVersion(0)
.setReplicas(partition1Replicas)
.setIsNew(true)
).asJava,
Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build()
@ -1187,11 +1336,27 @@ class ReplicaManagerTest {
// make broker 0 the leader of partition 1 so broker 1 loses its leadership position
val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId,
controllerEpoch, brokerEpoch,
collection.immutable.Map(
tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement,
partition0Replicas, 0, partition0Replicas, true),
tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement,
partition1Replicas, 0, partition1Replicas, true)
Seq(
new LeaderAndIsrPartitionState()
.setTopicName(tp0.topic)
.setPartitionIndex(tp0.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(0)
.setLeaderEpoch(leaderEpoch + leaderEpochIncrement)
.setIsr(partition0Replicas)
.setZkVersion(0)
.setReplicas(partition0Replicas)
.setIsNew(true),
new LeaderAndIsrPartitionState()
.setTopicName(tp1.topic)
.setPartitionIndex(tp1.partition)
.setControllerEpoch(controllerEpoch)
.setLeader(0)
.setLeaderEpoch(leaderEpoch + leaderEpochIncrement)
.setIsr(partition1Replicas)
.setZkVersion(0)
.setReplicas(partition1Replicas)
.setIsNew(true)
).asJava,
Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build()

View File

@ -27,7 +27,9 @@ import org.apache.kafka.common.acl._
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection}
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.message._
import org.apache.kafka.common.metrics.{KafkaMetric, Quota, Sensor}
import org.apache.kafka.common.network.ListenerName
@ -227,20 +229,39 @@ class RequestQuotaTest extends BaseRequestTest {
case ApiKeys.LEADER_AND_ISR =>
new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Long.MaxValue,
Map(tp -> new LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava,
2, Seq(brokerId).asJava, true)).asJava,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(Int.MaxValue)
.setLeader(brokerId)
.setLeaderEpoch(Int.MaxValue)
.setIsr(List(brokerId).asJava)
.setZkVersion(2)
.setReplicas(Seq(brokerId).asJava)
.setIsNew(true)).asJava,
Set(new Node(brokerId, "localhost", 0)).asJava)
case ApiKeys.STOP_REPLICA =>
new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId, Int.MaxValue, Long.MaxValue, true, Set(tp).asJava)
case ApiKeys.UPDATE_METADATA =>
val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState(
Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava
val partitionState = Seq(new UpdateMetadataPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(Int.MaxValue)
.setLeader(brokerId)
.setLeaderEpoch(Int.MaxValue)
.setIsr(List(brokerId).asJava)
.setZkVersion(2)
.setReplicas(Seq(brokerId).asJava)).asJava
val securityProtocol = SecurityProtocol.PLAINTEXT
val brokers = Set(new UpdateMetadataRequest.Broker(brokerId,
Seq(new UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol,
ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava
val brokers = Seq(new UpdateMetadataBroker()
.setId(brokerId)
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("localhost")
.setPort(0)
.setSecurityProtocol(securityProtocol.id)
.setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)).asJava
new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, brokerId, Int.MaxValue, Long.MaxValue, partitionState, brokers)
case ApiKeys.CONTROLLED_SHUTDOWN =>

View File

@ -233,7 +233,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness {
// Initiate a sendRequest and wait until connection is established and one byte is received by the peer
val requestBuilder = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion,
controllerId, 1, 0L, Map.empty.asJava, brokerAndEpochs.keys.map(_.node(listenerName)).toSet.asJava)
controllerId, 1, 0L, Seq.empty.asJava, brokerAndEpochs.keys.map(_.node(listenerName)).toSet.asJava)
controllerChannelManager.sendRequest(1, requestBuilder)
receiveFuture.get(10, TimeUnit.SECONDS)

View File

@ -25,7 +25,6 @@ import org.junit.Assert._
import org.junit.Test
import collection.JavaConverters._
class StopReplicaRequestTest extends BaseRequestTest {
override val logDirCount = 2
override val brokerCount: Int = 1
@ -44,14 +43,16 @@ class StopReplicaRequestTest extends BaseRequestTest {
val offlineDir = server.logManager.getLog(tp1).get.dir.getParent
server.replicaManager.handleLogDirFailure(offlineDir, sendZkNotification = false)
for (i <- 1 to 2) {
for (_ <- 1 to 2) {
val request1 = new StopReplicaRequest.Builder(1,
server.config.brokerId, server.replicaManager.controllerEpoch, server.kafkaController.brokerEpoch,
true, Set(tp0, tp1).asJava).build()
val response1 = connectAndSend(request1, ApiKeys.STOP_REPLICA, controllerSocketServer)
val partitionErrors1 = StopReplicaResponse.parse(response1, request1.version).responses()
assertEquals(Errors.NONE, partitionErrors1.get(tp0))
assertEquals(Errors.KAFKA_STORAGE_ERROR, partitionErrors1.get(tp1))
val partitionErrors1 = StopReplicaResponse.parse(response1, request1.version).partitionErrors.asScala
assertEquals(Some(Errors.NONE.code),
partitionErrors1.find(pe => pe.topicName == tp0.topic && pe.partitionIndex == tp0.partition).map(_.errorCode))
assertEquals(Some(Errors.KAFKA_STORAGE_ERROR.code),
partitionErrors1.find(pe => pe.topicName == tp1.topic && pe.partitionIndex == tp1.partition).map(_.errorCode))
}
}

View File

@ -231,10 +231,7 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging {
private def waitForEpochChangeTo(topic: String, partition: Int, epoch: Int): Unit = {
TestUtils.waitUntilTrue(() => {
brokers(0).metadataCache.getPartitionInfo(topic, partition) match {
case Some(m) => m.basePartitionState.leaderEpoch == epoch
case None => false
}
brokers(0).metadataCache.getPartitionInfo(topic, partition).exists(_.leaderEpoch == epoch)
}, "Epoch didn't change")
}

View File

@ -910,16 +910,14 @@ object TestUtils extends Logging {
def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int,
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
var leader: Int = -1
waitUntilTrue(() =>
servers.foldLeft(true) {
(result, server) =>
val partitionStateOpt = server.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic, partition)
partitionStateOpt match {
case None => false
case Some(partitionState) =>
leader = partitionState.basePartitionState.leader
result && Request.isValidBrokerId(leader)
}
waitUntilTrue(
() => servers.forall { server =>
server.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic, partition) match {
case Some(partitionState) if Request.isValidBrokerId(partitionState.leader) =>
leader = partitionState.leader
true
case _ => false
}
},
"Partition [%s,%d] metadata not propagated after %d ms".format(topic, partition, timeout),
waitTimeMs = timeout)

View File

@ -33,7 +33,7 @@ import kafka.server.MetadataCache;
import kafka.server.checkpoints.OffsetCheckpoints;
import kafka.utils.KafkaScheduler;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
import org.apache.kafka.common.utils.Time;
import org.mockito.Mockito;
import org.openjdk.jmh.annotations.Benchmark;
@ -54,7 +54,6 @@ import scala.collection.JavaConverters;
import java.io.File;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Properties;
import java.util.UUID;
@ -106,8 +105,14 @@ public class UpdateFollowerFetchStateBenchmark {
replicas.add(0);
replicas.add(1);
replicas.add(2);
LeaderAndIsrRequest.PartitionState partitionState = new LeaderAndIsrRequest.PartitionState(
0, 0, 0, replicas, 1, replicas, true);
LeaderAndIsrPartitionState partitionState = new LeaderAndIsrPartitionState()
.setControllerEpoch(0)
.setLeader(0)
.setLeaderEpoch(0)
.setIsr(replicas)
.setZkVersion(1)
.setReplicas(replicas)
.setIsNew(true);
PartitionStateStore partitionStateStore = Mockito.mock(PartitionStateStore.class);
Mockito.when(partitionStateStore.fetchTopicConfig()).thenReturn(new Properties());
partition = new Partition(topicPartition, 100,

View File

@ -31,7 +31,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.requests.UpdateMetadataRequest;
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.KafkaStreams;
@ -705,13 +705,13 @@ public class IntegrationTestUtils {
TestUtils.waitForCondition(() -> {
for (final KafkaServer server : servers) {
final MetadataCache metadataCache = server.dataPlaneRequestProcessor().metadataCache();
final Option<UpdateMetadataRequest.PartitionState> partitionInfo =
final Option<UpdateMetadataPartitionState> partitionInfo =
metadataCache.getPartitionInfo(topic, partition);
if (partitionInfo.isEmpty()) {
return false;
}
final UpdateMetadataRequest.PartitionState metadataPartitionState = partitionInfo.get();
if (!Request.isValidBrokerId(metadataPartitionState.basePartitionState.leader)) {
final UpdateMetadataPartitionState metadataPartitionState = partitionInfo.get();
if (!Request.isValidBrokerId(metadataPartitionState.leader())) {
return false;
}
}