mirror of https://github.com/apache/kafka.git
KAFKA-14552: Assume a baseline of 3.0 for server protocol versions (#18497)
Kafka 4.0 will remove support for zk mode and will require conversion to kraft before upgrading to 4.0. The minimum kraft version is 3.0 (aka 3.0-IV1). This provides an opportunity to remove exclusively server side protocols versions that only exist to allow direct upgrades from versions older than 3.0 or that are used only by zk mode. Since KRaft became production ready in 3.3, we should consider setting the baseline to 3.3. But that requires more discussion and it can be done via a separate change (KAFKA-18601). Protocol changes: * Remove RequestHeader v0 (only used by ControlledShutdown v0) * Remove WriteTxnMarkers v0 * Remove all versions of ControlledShutdown, LeaderAndIsr, StopReplica, UpdateMetadata In order to remove all versions safely, extend generator to support setting "versions" to "none". In this case, we no longer generate the `*Data` classes, but we still reserve the id for the relevant protocol api (so it doesn't get accidentally used for something else). The protocol documentation is correct after these changes. We kept a simplified version of `LeaderAndIsr{Request|Response}` because it's used by many tests that are still relevant in kraft mode. Once KAFKA-18486 is done, it may be possible to remove it (I left a comment on the ticket). Similarly, KAFKA-18487 may make it possible to remove the introduced `StopReplicaPartitionState` (left a comment on that ticket too). There are a number of places that were adjusted to include an `ApiKeys.hasValidVersion` check. Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
837fb1ed02
commit
87b37a4065
|
@ -255,6 +255,15 @@ public enum ApiKeys {
|
|||
return apiVersion >= messageType.lowestDeprecatedVersion() && apiVersion <= messageType.highestDeprecatedVersion();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns `true` if there is at least one valid version, `false` otherwise. When `false` is returned, it typically
|
||||
* means that the protocol api is no longer supported, but the api key remains assigned to the removed api so we
|
||||
* do not accidentally reuse it for a different api.
|
||||
*/
|
||||
public boolean hasValidVersion() {
|
||||
return oldestVersion() <= latestVersion();
|
||||
}
|
||||
|
||||
public Optional<ApiVersionsResponseData.ApiVersion> toApiVersion(boolean enableUnstableLastVersion) {
|
||||
short oldestVersion = oldestVersion();
|
||||
short latestVersion = latestVersion(enableUnstableLastVersion);
|
||||
|
|
|
@ -153,14 +153,14 @@ public class Protocol {
|
|||
final StringBuilder b = new StringBuilder();
|
||||
b.append("<h5>Headers:</h5>\n");
|
||||
|
||||
for (int i = 0; i < RequestHeaderData.SCHEMAS.length; i++) {
|
||||
for (int i = RequestHeaderData.LOWEST_SUPPORTED_VERSION; i <= RequestHeaderData.HIGHEST_SUPPORTED_VERSION; i++) {
|
||||
b.append("<pre>");
|
||||
b.append("Request Header v").append(i).append(" => ");
|
||||
schemaToBnfHtml(RequestHeaderData.SCHEMAS[i], b, 2);
|
||||
b.append("</pre>\n");
|
||||
schemaToFieldTableHtml(RequestHeaderData.SCHEMAS[i], b);
|
||||
}
|
||||
for (int i = 0; i < ResponseHeaderData.SCHEMAS.length; i++) {
|
||||
for (int i = ResponseHeaderData.LOWEST_SUPPORTED_VERSION; i <= ResponseHeaderData.HIGHEST_SUPPORTED_VERSION; i++) {
|
||||
b.append("<pre>");
|
||||
b.append("Response Header v").append(i).append(" => ");
|
||||
schemaToBnfHtml(ResponseHeaderData.SCHEMAS[i], b, 2);
|
||||
|
@ -178,30 +178,30 @@ public class Protocol {
|
|||
// Requests
|
||||
b.append("<b>Requests:</b><br>\n");
|
||||
Schema[] requests = key.messageType.requestSchemas();
|
||||
for (int i = 0; i < requests.length; i++) {
|
||||
Schema schema = requests[i];
|
||||
for (short version = key.oldestVersion(); version <= key.latestVersion(); version++) {
|
||||
Schema schema = requests[version];
|
||||
if (schema == null)
|
||||
throw new IllegalStateException("Unexpected null schema for " + key + " with version " + version);
|
||||
// Schema
|
||||
if (schema != null) {
|
||||
b.append("<div>");
|
||||
// Version header
|
||||
b.append("<pre>");
|
||||
b.append(key.name);
|
||||
b.append(" Request (Version: ");
|
||||
b.append(i);
|
||||
b.append(") => ");
|
||||
schemaToBnfHtml(requests[i], b, 2);
|
||||
b.append("</pre>");
|
||||
b.append("<div>");
|
||||
// Version header
|
||||
b.append("<pre>");
|
||||
b.append(key.name);
|
||||
b.append(" Request (Version: ");
|
||||
b.append(version);
|
||||
b.append(") => ");
|
||||
schemaToBnfHtml(schema, b, 2);
|
||||
b.append("</pre>");
|
||||
|
||||
if (!key.isVersionEnabled((short) i, false)) {
|
||||
b.append("<p>This version of the request is unstable.</p>");
|
||||
}
|
||||
|
||||
b.append("<p><b>Request header version:</b> ");
|
||||
b.append(key.requestHeaderVersion((short) i));
|
||||
b.append("</p>\n");
|
||||
|
||||
schemaToFieldTableHtml(requests[i], b);
|
||||
if (!key.isVersionEnabled(version, false)) {
|
||||
b.append("<p>This version of the request is unstable.</p>");
|
||||
}
|
||||
|
||||
b.append("<p><b>Request header version:</b> ");
|
||||
b.append(key.requestHeaderVersion(version));
|
||||
b.append("</p>\n");
|
||||
|
||||
schemaToFieldTableHtml(schema, b);
|
||||
b.append("</div>\n");
|
||||
}
|
||||
|
||||
|
|
|
@ -1,98 +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.protocol.ApiKeys;
|
||||
|
||||
// Abstract class for all control requests including UpdateMetadataRequest, LeaderAndIsrRequest and StopReplicaRequest
|
||||
public abstract class AbstractControlRequest extends AbstractRequest {
|
||||
|
||||
/**
|
||||
* Indicates if a controller request is incremental, full, or unknown.
|
||||
* Used by LeaderAndIsrRequest.Type and UpdateMetadataRequest.Type fields.
|
||||
*/
|
||||
public enum Type {
|
||||
UNKNOWN(0),
|
||||
INCREMENTAL(1),
|
||||
FULL(2);
|
||||
|
||||
private final byte type;
|
||||
Type(int type) {
|
||||
this.type = (byte) type;
|
||||
}
|
||||
|
||||
public byte toByte() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public static Type fromByte(byte type) {
|
||||
for (Type t : Type.values()) {
|
||||
if (t.type == type) {
|
||||
return t;
|
||||
}
|
||||
}
|
||||
return UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
||||
public static final long UNKNOWN_BROKER_EPOCH = -1L;
|
||||
|
||||
public abstract static class Builder<T extends AbstractRequest> extends AbstractRequest.Builder<T> {
|
||||
protected final int controllerId;
|
||||
protected final int controllerEpoch;
|
||||
protected final long brokerEpoch;
|
||||
protected final boolean kraftController;
|
||||
|
||||
protected Builder(ApiKeys api, short version, int controllerId, int controllerEpoch, long brokerEpoch) {
|
||||
this(api, version, controllerId, controllerEpoch, brokerEpoch, false);
|
||||
}
|
||||
|
||||
protected Builder(ApiKeys api, short version, int controllerId, int controllerEpoch,
|
||||
long brokerEpoch, boolean kraftController) {
|
||||
super(api, version);
|
||||
this.controllerId = controllerId;
|
||||
this.controllerEpoch = controllerEpoch;
|
||||
this.brokerEpoch = brokerEpoch;
|
||||
this.kraftController = kraftController;
|
||||
}
|
||||
|
||||
public int controllerId() {
|
||||
return controllerId;
|
||||
}
|
||||
|
||||
public int controllerEpoch() {
|
||||
return controllerEpoch;
|
||||
}
|
||||
|
||||
public long brokerEpoch() {
|
||||
return brokerEpoch;
|
||||
}
|
||||
}
|
||||
|
||||
protected AbstractControlRequest(ApiKeys api, short version) {
|
||||
super(api, version);
|
||||
}
|
||||
|
||||
public abstract int controllerId();
|
||||
|
||||
public abstract boolean isKRaftController();
|
||||
|
||||
public abstract int controllerEpoch();
|
||||
|
||||
public abstract long brokerEpoch();
|
||||
|
||||
}
|
|
@ -196,14 +196,6 @@ public abstract class AbstractRequest implements AbstractRequestResponse {
|
|||
return LeaveGroupRequest.parse(buffer, apiVersion);
|
||||
case SYNC_GROUP:
|
||||
return SyncGroupRequest.parse(buffer, apiVersion);
|
||||
case STOP_REPLICA:
|
||||
return StopReplicaRequest.parse(buffer, apiVersion);
|
||||
case CONTROLLED_SHUTDOWN:
|
||||
return ControlledShutdownRequest.parse(buffer, apiVersion);
|
||||
case UPDATE_METADATA:
|
||||
return UpdateMetadataRequest.parse(buffer, apiVersion);
|
||||
case LEADER_AND_ISR:
|
||||
return LeaderAndIsrRequest.parse(buffer, apiVersion);
|
||||
case DESCRIBE_GROUPS:
|
||||
return DescribeGroupsRequest.parse(buffer, apiVersion);
|
||||
case LIST_GROUPS:
|
||||
|
|
|
@ -62,29 +62,29 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
|
|||
*/
|
||||
public abstract Map<Errors, Integer> errorCounts();
|
||||
|
||||
protected Map<Errors, Integer> errorCounts(Errors error) {
|
||||
protected static Map<Errors, Integer> errorCounts(Errors error) {
|
||||
return Collections.singletonMap(error, 1);
|
||||
}
|
||||
|
||||
protected Map<Errors, Integer> errorCounts(Stream<Errors> errors) {
|
||||
protected static Map<Errors, Integer> errorCounts(Stream<Errors> errors) {
|
||||
return errors.collect(Collectors.groupingBy(e -> e, Collectors.summingInt(e -> 1)));
|
||||
}
|
||||
|
||||
protected Map<Errors, Integer> errorCounts(Collection<Errors> errors) {
|
||||
protected static Map<Errors, Integer> errorCounts(Collection<Errors> errors) {
|
||||
Map<Errors, Integer> errorCounts = new HashMap<>();
|
||||
for (Errors error : errors)
|
||||
updateErrorCounts(errorCounts, error);
|
||||
return errorCounts;
|
||||
}
|
||||
|
||||
protected Map<Errors, Integer> apiErrorCounts(Map<?, ApiError> errors) {
|
||||
protected static Map<Errors, Integer> apiErrorCounts(Map<?, ApiError> errors) {
|
||||
Map<Errors, Integer> errorCounts = new HashMap<>();
|
||||
for (ApiError apiError : errors.values())
|
||||
updateErrorCounts(errorCounts, apiError.error());
|
||||
return errorCounts;
|
||||
}
|
||||
|
||||
protected void updateErrorCounts(Map<Errors, Integer> errorCounts, Errors error) {
|
||||
protected static void updateErrorCounts(Map<Errors, Integer> errorCounts, Errors error) {
|
||||
Integer count = errorCounts.getOrDefault(error, 0);
|
||||
errorCounts.put(error, count + 1);
|
||||
}
|
||||
|
@ -133,14 +133,6 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
|
|||
return LeaveGroupResponse.parse(responseBuffer, version);
|
||||
case SYNC_GROUP:
|
||||
return SyncGroupResponse.parse(responseBuffer, version);
|
||||
case STOP_REPLICA:
|
||||
return StopReplicaResponse.parse(responseBuffer, version);
|
||||
case CONTROLLED_SHUTDOWN:
|
||||
return ControlledShutdownResponse.parse(responseBuffer, version);
|
||||
case UPDATE_METADATA:
|
||||
return UpdateMetadataResponse.parse(responseBuffer, version);
|
||||
case LEADER_AND_ISR:
|
||||
return LeaderAndIsrResponse.parse(responseBuffer, version);
|
||||
case DESCRIBE_GROUPS:
|
||||
return DescribeGroupsResponse.parse(responseBuffer, version);
|
||||
case LIST_GROUPS:
|
||||
|
|
|
@ -1,72 +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.message.ControlledShutdownRequestData;
|
||||
import org.apache.kafka.common.message.ControlledShutdownResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class ControlledShutdownRequest extends AbstractRequest {
|
||||
|
||||
public static class Builder extends AbstractRequest.Builder<ControlledShutdownRequest> {
|
||||
|
||||
private final ControlledShutdownRequestData data;
|
||||
|
||||
public Builder(ControlledShutdownRequestData data, short desiredVersion) {
|
||||
super(ApiKeys.CONTROLLED_SHUTDOWN, desiredVersion);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControlledShutdownRequest build(short version) {
|
||||
return new ControlledShutdownRequest(data, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return data.toString();
|
||||
}
|
||||
}
|
||||
|
||||
private final ControlledShutdownRequestData data;
|
||||
|
||||
private ControlledShutdownRequest(ControlledShutdownRequestData data, short version) {
|
||||
super(ApiKeys.CONTROLLED_SHUTDOWN, version);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControlledShutdownResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
ControlledShutdownResponseData data = new ControlledShutdownResponseData()
|
||||
.setErrorCode(Errors.forException(e).code());
|
||||
return new ControlledShutdownResponse(data);
|
||||
}
|
||||
|
||||
public static ControlledShutdownRequest parse(ByteBuffer buffer, short version) {
|
||||
return new ControlledShutdownRequest(new ControlledShutdownRequestData(new ByteBufferAccessor(buffer), version),
|
||||
version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControlledShutdownRequestData data() {
|
||||
return data;
|
||||
}
|
||||
}
|
|
@ -1,88 +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.apache.kafka.common.message.ControlledShutdownResponseData;
|
||||
import org.apache.kafka.common.message.ControlledShutdownResponseData.RemainingPartition;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
public class ControlledShutdownResponse extends AbstractResponse {
|
||||
|
||||
/**
|
||||
* Possible error codes:
|
||||
*
|
||||
* UNKNOWN(-1) (this is because IllegalStateException may be thrown in `KafkaController.shutdownBroker`, it would be good to improve this)
|
||||
* BROKER_NOT_AVAILABLE(8)
|
||||
* STALE_CONTROLLER_EPOCH(11)
|
||||
*/
|
||||
private final ControlledShutdownResponseData data;
|
||||
|
||||
public ControlledShutdownResponse(ControlledShutdownResponseData data) {
|
||||
super(ApiKeys.CONTROLLED_SHUTDOWN);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
public Errors error() {
|
||||
return Errors.forCode(data.errorCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
return errorCounts(error());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int throttleTimeMs() {
|
||||
return DEFAULT_THROTTLE_TIME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maybeSetThrottleTimeMs(int throttleTimeMs) {
|
||||
// Not supported by the response schema
|
||||
}
|
||||
|
||||
public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) {
|
||||
return new ControlledShutdownResponse(new ControlledShutdownResponseData(new ByteBufferAccessor(buffer), version));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControlledShutdownResponseData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
public static ControlledShutdownResponse prepareResponse(Errors error, Set<TopicPartition> tps) {
|
||||
ControlledShutdownResponseData data = new ControlledShutdownResponseData();
|
||||
data.setErrorCode(error.code());
|
||||
ControlledShutdownResponseData.RemainingPartitionCollection pSet = new ControlledShutdownResponseData.RemainingPartitionCollection();
|
||||
tps.forEach(tp ->
|
||||
pSet.add(new RemainingPartition()
|
||||
.setTopicName(tp.topic())
|
||||
.setPartitionIndex(tp.partition()))
|
||||
);
|
||||
data.setRemainingPartitions(pSet);
|
||||
return new ControlledShutdownResponse(data);
|
||||
}
|
||||
|
||||
}
|
|
@ -18,96 +18,69 @@ package org.apache.kafka.common.requests;
|
|||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
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.message.LeaderAndIsrRequestData.LeaderAndIsrTopicState;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError;
|
||||
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.MessageUtil;
|
||||
import org.apache.kafka.common.utils.FlattenedIterator;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public final class LeaderAndIsrRequest extends AbstractControlRequest {
|
||||
public final class LeaderAndIsrRequest {
|
||||
|
||||
public static class Builder extends AbstractControlRequest.Builder<LeaderAndIsrRequest> {
|
||||
/**
|
||||
* Indicates if a controller request is incremental, full, or unknown.
|
||||
*/
|
||||
public enum Type {
|
||||
UNKNOWN(0),
|
||||
INCREMENTAL(1),
|
||||
FULL(2);
|
||||
|
||||
private final List<LeaderAndIsrPartitionState> partitionStates;
|
||||
private final byte type;
|
||||
Type(int type) {
|
||||
this.type = (byte) type;
|
||||
}
|
||||
|
||||
public byte toByte() {
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
protected final int controllerId;
|
||||
protected final int controllerEpoch;
|
||||
protected final long brokerEpoch;
|
||||
private final List<PartitionState> partitionStates;
|
||||
private final Map<String, Uuid> topicIds;
|
||||
private final Collection<Node> liveLeaders;
|
||||
private final Type updateType;
|
||||
|
||||
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
List<LeaderAndIsrPartitionState> partitionStates, Map<String, Uuid> topicIds,
|
||||
public Builder(int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
List<PartitionState> partitionStates, Map<String, Uuid> topicIds,
|
||||
Collection<Node> liveLeaders) {
|
||||
this(version, controllerId, controllerEpoch, brokerEpoch, partitionStates, topicIds,
|
||||
liveLeaders, false, Type.UNKNOWN);
|
||||
this(controllerId, controllerEpoch, brokerEpoch, partitionStates, topicIds, liveLeaders, Type.UNKNOWN);
|
||||
}
|
||||
|
||||
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
List<LeaderAndIsrPartitionState> partitionStates, Map<String, Uuid> topicIds,
|
||||
Collection<Node> liveLeaders, boolean kraftController, Type updateType) {
|
||||
super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch, kraftController);
|
||||
public Builder(int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
List<PartitionState> partitionStates, Map<String, Uuid> topicIds,
|
||||
Collection<Node> liveLeaders, Type updateType) {
|
||||
this.controllerId = controllerId;
|
||||
this.controllerEpoch = controllerEpoch;
|
||||
this.brokerEpoch = brokerEpoch;
|
||||
this.partitionStates = partitionStates;
|
||||
this.topicIds = topicIds;
|
||||
this.liveLeaders = liveLeaders;
|
||||
this.updateType = updateType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeaderAndIsrRequest build(short 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 >= 7) {
|
||||
data.setIsKRaftController(kraftController);
|
||||
}
|
||||
|
||||
if (version >= 5) {
|
||||
data.setType(updateType.toByte());
|
||||
}
|
||||
|
||||
if (version >= 2) {
|
||||
Map<String, LeaderAndIsrTopicState> topicStatesMap = groupByTopic(partitionStates, topicIds);
|
||||
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, Uuid> topicIds) {
|
||||
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())
|
||||
.setTopicId(topicIds.getOrDefault(partition.topicName(), Uuid.ZERO_UUID)));
|
||||
topicState.partitionStates().add(partition);
|
||||
}
|
||||
return topicStates;
|
||||
public LeaderAndIsrRequest build() {
|
||||
return new LeaderAndIsrRequest(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -124,105 +97,307 @@ public final class LeaderAndIsrRequest extends AbstractControlRequest {
|
|||
}
|
||||
}
|
||||
|
||||
private final LeaderAndIsrRequestData data;
|
||||
private final int controllerId;
|
||||
private final int controllerEpoch;
|
||||
private final long brokerEpoch;
|
||||
private final List<Node> liveLeaders;
|
||||
private final List<TopicState> topicStates;
|
||||
private final Type requestType;
|
||||
|
||||
public 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();
|
||||
public LeaderAndIsrRequest(Builder builder) {
|
||||
this.controllerId = builder.controllerId;
|
||||
this.controllerEpoch = builder.controllerEpoch;
|
||||
this.brokerEpoch = builder.brokerEpoch;
|
||||
this.requestType = builder.updateType;
|
||||
this.liveLeaders = new ArrayList<>(builder.liveLeaders);
|
||||
this.topicStates = new ArrayList<>(groupByTopic(builder.partitionStates, builder.topicIds).values());
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
}
|
||||
private static Map<String, TopicState> groupByTopic(List<PartitionState> partitionStates, Map<String, Uuid> topicIds) {
|
||||
Map<String, TopicState> topicStates = new HashMap<>();
|
||||
for (PartitionState partition : partitionStates) {
|
||||
TopicState topicState = topicStates.computeIfAbsent(partition.topicName(), t -> {
|
||||
var topic = new TopicState();
|
||||
topic.topicName = partition.topicName();
|
||||
topic.topicId = topicIds.getOrDefault(partition.topicName(), Uuid.ZERO_UUID);
|
||||
return topic;
|
||||
});
|
||||
topicState.partitionStates().add(partition);
|
||||
}
|
||||
return topicStates;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeaderAndIsrResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
LeaderAndIsrResponseData responseData = new LeaderAndIsrResponseData();
|
||||
Errors error = Errors.forException(e);
|
||||
responseData.setErrorCode(error.code());
|
||||
|
||||
if (version() < 5) {
|
||||
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);
|
||||
} else {
|
||||
for (LeaderAndIsrTopicState topicState : data.topicStates()) {
|
||||
List<LeaderAndIsrPartitionError> partitions = new ArrayList<>(
|
||||
topicState.partitionStates().size());
|
||||
for (LeaderAndIsrPartitionState partition : topicState.partitionStates()) {
|
||||
partitions.add(new LeaderAndIsrPartitionError()
|
||||
.setPartitionIndex(partition.partitionIndex())
|
||||
.setErrorCode(error.code()));
|
||||
}
|
||||
responseData.topics().add(new LeaderAndIsrTopicError()
|
||||
.setTopicId(topicState.topicId())
|
||||
.setPartitionErrors(partitions));
|
||||
}
|
||||
}
|
||||
|
||||
return new LeaderAndIsrResponse(responseData, version());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int controllerId() {
|
||||
return data.controllerId();
|
||||
return controllerId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isKRaftController() {
|
||||
return data.isKRaftController();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int controllerEpoch() {
|
||||
return data.controllerEpoch();
|
||||
return controllerEpoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long brokerEpoch() {
|
||||
return data.brokerEpoch();
|
||||
return brokerEpoch;
|
||||
}
|
||||
|
||||
public Iterable<LeaderAndIsrPartitionState> partitionStates() {
|
||||
if (version() >= 2)
|
||||
return () -> new FlattenedIterator<>(data.topicStates().iterator(),
|
||||
public Iterable<PartitionState> partitionStates() {
|
||||
return () -> new FlattenedIterator<>(topicStates.iterator(),
|
||||
topicState -> topicState.partitionStates().iterator());
|
||||
return data.ungroupedPartitionStates();
|
||||
}
|
||||
|
||||
public Map<String, Uuid> topicIds() {
|
||||
return data.topicStates().stream()
|
||||
.collect(Collectors.toMap(LeaderAndIsrTopicState::topicName, LeaderAndIsrTopicState::topicId));
|
||||
return topicStates.stream()
|
||||
.collect(Collectors.toMap(TopicState::topicName, TopicState::topicId));
|
||||
}
|
||||
|
||||
public List<LeaderAndIsrLiveLeader> liveLeaders() {
|
||||
return Collections.unmodifiableList(data.liveLeaders());
|
||||
public List<Node> liveLeaders() {
|
||||
return Collections.unmodifiableList(liveLeaders);
|
||||
}
|
||||
|
||||
public Type requestType() {
|
||||
return Type.fromByte(data.type());
|
||||
return requestType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeaderAndIsrRequestData data() {
|
||||
return data;
|
||||
public LeaderAndIsrResponse getErrorResponse(Exception e) {
|
||||
LinkedHashMap<Uuid, List<LeaderAndIsrResponse.PartitionError>> errorsMap = new LinkedHashMap<>();
|
||||
Errors error = Errors.forException(e);
|
||||
|
||||
for (TopicState topicState : topicStates) {
|
||||
List<LeaderAndIsrResponse.PartitionError> partitions = new ArrayList<>(topicState.partitionStates().size());
|
||||
for (PartitionState partition : topicState.partitionStates()) {
|
||||
partitions.add(new LeaderAndIsrResponse.PartitionError(partition.partitionIndex, error.code()));
|
||||
}
|
||||
errorsMap.put(topicState.topicId, partitions);
|
||||
}
|
||||
|
||||
return new LeaderAndIsrResponse(error, errorsMap);
|
||||
|
||||
}
|
||||
|
||||
public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) {
|
||||
return new LeaderAndIsrRequest(new LeaderAndIsrRequestData(new ByteBufferAccessor(buffer), version), version);
|
||||
public static class TopicState {
|
||||
String topicName;
|
||||
Uuid topicId;
|
||||
List<PartitionState> partitionStates;
|
||||
|
||||
public TopicState() {
|
||||
this.topicName = "";
|
||||
this.topicId = Uuid.ZERO_UUID;
|
||||
this.partitionStates = new ArrayList<>(0);
|
||||
}
|
||||
|
||||
public String topicName() {
|
||||
return this.topicName;
|
||||
}
|
||||
|
||||
public Uuid topicId() {
|
||||
return this.topicId;
|
||||
}
|
||||
|
||||
public List<PartitionState> partitionStates() {
|
||||
return this.partitionStates;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
TopicState that = (TopicState) o;
|
||||
return Objects.equals(topicName, that.topicName) &&
|
||||
Objects.equals(topicId, that.topicId) &&
|
||||
Objects.equals(partitionStates, that.partitionStates);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(topicName, topicId, partitionStates);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LeaderAndIsrTopicState("
|
||||
+ "topicName='" + topicName + "'"
|
||||
+ ", topicId=" + topicId
|
||||
+ ", partitionStates=" + MessageUtil.deepToString(partitionStates.iterator())
|
||||
+ ")";
|
||||
}
|
||||
}
|
||||
|
||||
public static class PartitionState {
|
||||
String topicName;
|
||||
int partitionIndex;
|
||||
int controllerEpoch;
|
||||
int leader;
|
||||
int leaderEpoch;
|
||||
List<Integer> isr;
|
||||
int partitionEpoch;
|
||||
List<Integer> replicas;
|
||||
List<Integer> addingReplicas;
|
||||
List<Integer> removingReplicas;
|
||||
boolean isNew;
|
||||
byte leaderRecoveryState;
|
||||
|
||||
public PartitionState() {
|
||||
this.topicName = "";
|
||||
this.partitionIndex = 0;
|
||||
this.controllerEpoch = 0;
|
||||
this.leader = 0;
|
||||
this.leaderEpoch = 0;
|
||||
this.isr = new ArrayList<>(0);
|
||||
this.partitionEpoch = 0;
|
||||
this.replicas = new ArrayList<>(0);
|
||||
this.addingReplicas = new ArrayList<>(0);
|
||||
this.removingReplicas = new ArrayList<>(0);
|
||||
this.isNew = false;
|
||||
this.leaderRecoveryState = (byte) 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
PartitionState that = (PartitionState) o;
|
||||
return partitionIndex == that.partitionIndex &&
|
||||
controllerEpoch == that.controllerEpoch &&
|
||||
leader == that.leader &&
|
||||
leaderEpoch == that.leaderEpoch &&
|
||||
partitionEpoch == that.partitionEpoch &&
|
||||
isNew == that.isNew &&
|
||||
leaderRecoveryState == that.leaderRecoveryState &&
|
||||
Objects.equals(topicName, that.topicName) &&
|
||||
Objects.equals(isr, that.isr) &&
|
||||
Objects.equals(replicas, that.replicas) &&
|
||||
Objects.equals(addingReplicas, that.addingReplicas) &&
|
||||
Objects.equals(removingReplicas, that.removingReplicas);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(topicName, partitionIndex, controllerEpoch, leader, leaderEpoch, isr, partitionEpoch,
|
||||
replicas, addingReplicas, removingReplicas, isNew, leaderRecoveryState);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LeaderAndIsrPartitionState("
|
||||
+ "topicName='" + topicName + "'"
|
||||
+ ", partitionIndex=" + partitionIndex
|
||||
+ ", controllerEpoch=" + controllerEpoch
|
||||
+ ", leader=" + leader
|
||||
+ ", leaderEpoch=" + leaderEpoch
|
||||
+ ", isr=" + MessageUtil.deepToString(isr.iterator())
|
||||
+ ", partitionEpoch=" + partitionEpoch
|
||||
+ ", replicas=" + MessageUtil.deepToString(replicas.iterator())
|
||||
+ ", addingReplicas=" + MessageUtil.deepToString(addingReplicas.iterator())
|
||||
+ ", removingReplicas=" + MessageUtil.deepToString(removingReplicas.iterator())
|
||||
+ ", isNew=" + (isNew ? "true" : "false")
|
||||
+ ", leaderRecoveryState=" + leaderRecoveryState
|
||||
+ ")";
|
||||
}
|
||||
|
||||
public String topicName() {
|
||||
return this.topicName;
|
||||
}
|
||||
|
||||
public int partitionIndex() {
|
||||
return this.partitionIndex;
|
||||
}
|
||||
|
||||
public int controllerEpoch() {
|
||||
return this.controllerEpoch;
|
||||
}
|
||||
|
||||
public int leader() {
|
||||
return this.leader;
|
||||
}
|
||||
|
||||
public int leaderEpoch() {
|
||||
return this.leaderEpoch;
|
||||
}
|
||||
|
||||
public List<Integer> isr() {
|
||||
return this.isr;
|
||||
}
|
||||
|
||||
public int partitionEpoch() {
|
||||
return this.partitionEpoch;
|
||||
}
|
||||
|
||||
public List<Integer> replicas() {
|
||||
return this.replicas;
|
||||
}
|
||||
|
||||
public List<Integer> addingReplicas() {
|
||||
return this.addingReplicas;
|
||||
}
|
||||
|
||||
public List<Integer> removingReplicas() {
|
||||
return this.removingReplicas;
|
||||
}
|
||||
|
||||
public boolean isNew() {
|
||||
return this.isNew;
|
||||
}
|
||||
|
||||
public byte leaderRecoveryState() {
|
||||
return this.leaderRecoveryState;
|
||||
}
|
||||
|
||||
public PartitionState setTopicName(String v) {
|
||||
this.topicName = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setPartitionIndex(int v) {
|
||||
this.partitionIndex = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setControllerEpoch(int v) {
|
||||
this.controllerEpoch = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setLeader(int v) {
|
||||
this.leader = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setLeaderEpoch(int v) {
|
||||
this.leaderEpoch = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setIsr(List<Integer> v) {
|
||||
this.isr = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setPartitionEpoch(int v) {
|
||||
this.partitionEpoch = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setReplicas(List<Integer> v) {
|
||||
this.replicas = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setAddingReplicas(List<Integer> v) {
|
||||
this.addingReplicas = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setRemovingReplicas(List<Integer> v) {
|
||||
this.removingReplicas = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setIsNew(boolean v) {
|
||||
this.isNew = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
public PartitionState setLeaderRecoveryState(byte v) {
|
||||
this.leaderRecoveryState = v;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,19 +18,15 @@ package org.apache.kafka.common.requests;
|
|||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicErrorCollection;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class LeaderAndIsrResponse extends AbstractResponse {
|
||||
public class LeaderAndIsrResponse {
|
||||
|
||||
/**
|
||||
* Possible error code:
|
||||
|
@ -38,84 +34,62 @@ public class LeaderAndIsrResponse extends AbstractResponse {
|
|||
* STALE_CONTROLLER_EPOCH (11)
|
||||
* STALE_BROKER_EPOCH (77)
|
||||
*/
|
||||
private final LeaderAndIsrResponseData data;
|
||||
private final short version;
|
||||
private final Errors error;
|
||||
private final LinkedHashMap<Uuid, List<PartitionError>> topicErrors;
|
||||
|
||||
public LeaderAndIsrResponse(LeaderAndIsrResponseData data, short version) {
|
||||
super(ApiKeys.LEADER_AND_ISR);
|
||||
this.data = data;
|
||||
this.version = version;
|
||||
public LeaderAndIsrResponse(Errors error, LinkedHashMap<Uuid, List<PartitionError>> topicErrors) {
|
||||
this.error = error;
|
||||
this.topicErrors = topicErrors;
|
||||
}
|
||||
|
||||
public LeaderAndIsrTopicErrorCollection topics() {
|
||||
return this.data.topics();
|
||||
public LinkedHashMap<Uuid, List<PartitionError>> topics() {
|
||||
return topicErrors;
|
||||
}
|
||||
|
||||
public Errors error() {
|
||||
return Errors.forCode(data.errorCode());
|
||||
return error;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
Errors error = error();
|
||||
if (error != Errors.NONE) {
|
||||
// Minor optimization since the top-level error applies to all partitions
|
||||
if (version < 5)
|
||||
return Collections.singletonMap(error, data.partitionErrors().size() + 1);
|
||||
return Collections.singletonMap(error,
|
||||
data.topics().stream().mapToInt(t -> t.partitionErrors().size()).sum() + 1);
|
||||
return Collections.singletonMap(error, topics().values().stream().mapToInt(partitionErrors ->
|
||||
partitionErrors.size()).sum() + 1);
|
||||
}
|
||||
Map<Errors, Integer> errors;
|
||||
if (version < 5)
|
||||
errors = errorCounts(data.partitionErrors().stream().map(l -> Errors.forCode(l.errorCode())));
|
||||
else
|
||||
errors = errorCounts(data.topics().stream().flatMap(t -> t.partitionErrors().stream()).map(l ->
|
||||
Errors.forCode(l.errorCode())));
|
||||
updateErrorCounts(errors, Errors.NONE);
|
||||
Map<Errors, Integer> errors = AbstractResponse.errorCounts(topics().values().stream().flatMap(partitionErrors ->
|
||||
partitionErrors.stream()).map(p -> Errors.forCode(p.errorCode)));
|
||||
AbstractResponse.updateErrorCounts(errors, Errors.NONE);
|
||||
return errors;
|
||||
}
|
||||
|
||||
public Map<TopicPartition, Errors> partitionErrors(Map<Uuid, String> topicNames) {
|
||||
Map<TopicPartition, Errors> errors = new HashMap<>();
|
||||
if (version < 5) {
|
||||
data.partitionErrors().forEach(partition ->
|
||||
errors.put(new TopicPartition(partition.topicName(), partition.partitionIndex()),
|
||||
Errors.forCode(partition.errorCode())));
|
||||
} else {
|
||||
for (LeaderAndIsrTopicError topic : data.topics()) {
|
||||
String topicName = topicNames.get(topic.topicId());
|
||||
if (topicName != null) {
|
||||
topic.partitionErrors().forEach(partition ->
|
||||
errors.put(new TopicPartition(topicName, partition.partitionIndex()),
|
||||
Errors.forCode(partition.errorCode())));
|
||||
}
|
||||
topics().forEach((topicId, partitionErrors) -> {
|
||||
String topicName = topicNames.get(topicId);
|
||||
if (topicName != null) {
|
||||
partitionErrors.forEach(partition ->
|
||||
errors.put(new TopicPartition(topicName, partition.partitionIndex), Errors.forCode(partition.errorCode)));
|
||||
}
|
||||
}
|
||||
});
|
||||
return errors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int throttleTimeMs() {
|
||||
return DEFAULT_THROTTLE_TIME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maybeSetThrottleTimeMs(int throttleTimeMs) {
|
||||
// Not supported by the response schema
|
||||
}
|
||||
|
||||
public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) {
|
||||
return new LeaderAndIsrResponse(new LeaderAndIsrResponseData(new ByteBufferAccessor(buffer), version), version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeaderAndIsrResponseData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return data.toString();
|
||||
return "LeaderAndIsrResponse{" +
|
||||
"error=" + error +
|
||||
", topicErrors=" + topicErrors +
|
||||
'}';
|
||||
}
|
||||
|
||||
public static class PartitionError {
|
||||
public final int partitionIndex;
|
||||
public final short errorCode;
|
||||
|
||||
public PartitionError(int partitionIndex, short errorCode) {
|
||||
this.partitionIndex = partitionIndex;
|
||||
this.errorCode = errorCode;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -121,14 +121,21 @@ public class RequestHeader implements AbstractRequestResponse {
|
|||
}
|
||||
|
||||
public static RequestHeader parse(ByteBuffer buffer) {
|
||||
short apiKey = -1;
|
||||
short apiKeyId = -1;
|
||||
try {
|
||||
// We derive the header version from the request api version, so we read that first.
|
||||
// The request api version is part of `RequestHeaderData`, so we reset the buffer position after the read.
|
||||
int bufferStartPositionForHeader = buffer.position();
|
||||
apiKey = buffer.getShort();
|
||||
apiKeyId = buffer.getShort();
|
||||
short apiVersion = buffer.getShort();
|
||||
short headerVersion = ApiKeys.forId(apiKey).requestHeaderVersion(apiVersion);
|
||||
ApiKeys apiKey = ApiKeys.forId(apiKeyId);
|
||||
|
||||
// `apiKey.requestHeaderVersion` will fail if there are no valid versions - we do this check first in order to
|
||||
// provide a more helpful message
|
||||
if (!apiKey.hasValidVersion())
|
||||
throw new InvalidRequestException("Unsupported api with key " + apiKeyId + " (" + apiKey.name + ") and version " + apiVersion);
|
||||
|
||||
short headerVersion = apiKey.requestHeaderVersion(apiVersion);
|
||||
buffer.position(bufferStartPositionForHeader);
|
||||
final RequestHeaderData headerData = new RequestHeaderData(new ByteBufferAccessor(buffer), headerVersion);
|
||||
// Due to a quirk in the protocol, client ID is marked as nullable.
|
||||
|
@ -144,10 +151,12 @@ public class RequestHeader implements AbstractRequestResponse {
|
|||
header.size = Math.max(buffer.position() - bufferStartPositionForHeader, 0);
|
||||
return header;
|
||||
} catch (UnsupportedVersionException e) {
|
||||
throw new InvalidRequestException("Unknown API key " + apiKey, e);
|
||||
throw new InvalidRequestException("Unknown API key " + apiKeyId, e);
|
||||
} catch (InvalidRequestException e) {
|
||||
throw e;
|
||||
} catch (Throwable ex) {
|
||||
throw new InvalidRequestException("Error parsing request header. Our best guess of the apiKey is: " +
|
||||
apiKey, ex);
|
||||
throw new InvalidRequestException("Error parsing request header. Our best guess of the apiKeyId is: " +
|
||||
apiKeyId, ex);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,230 +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.apache.kafka.common.message.StopReplicaRequestData;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionV0;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopicState;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopicV1;
|
||||
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.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.utils.MappedIterator;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class StopReplicaRequest extends AbstractControlRequest {
|
||||
|
||||
public static class Builder extends AbstractControlRequest.Builder<StopReplicaRequest> {
|
||||
private final boolean deletePartitions;
|
||||
private final List<StopReplicaTopicState> topicStates;
|
||||
|
||||
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
boolean deletePartitions, List<StopReplicaTopicState> topicStates) {
|
||||
this(version, controllerId, controllerEpoch, brokerEpoch, deletePartitions,
|
||||
topicStates, false);
|
||||
}
|
||||
|
||||
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
boolean deletePartitions, List<StopReplicaTopicState> topicStates,
|
||||
boolean kraftController) {
|
||||
super(ApiKeys.STOP_REPLICA, version, controllerId, controllerEpoch, brokerEpoch, kraftController);
|
||||
this.deletePartitions = deletePartitions;
|
||||
this.topicStates = topicStates;
|
||||
}
|
||||
|
||||
public StopReplicaRequest build(short version) {
|
||||
StopReplicaRequestData data = new StopReplicaRequestData()
|
||||
.setControllerId(controllerId)
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setBrokerEpoch(brokerEpoch);
|
||||
|
||||
if (version >= 4) {
|
||||
data.setIsKRaftController(kraftController);
|
||||
}
|
||||
|
||||
if (version >= 3) {
|
||||
data.setTopicStates(topicStates);
|
||||
} else if (version >= 1) {
|
||||
data.setDeletePartitions(deletePartitions);
|
||||
List<StopReplicaTopicV1> topics = topicStates.stream().map(topic ->
|
||||
new StopReplicaTopicV1()
|
||||
.setName(topic.topicName())
|
||||
.setPartitionIndexes(topic.partitionStates().stream()
|
||||
.map(StopReplicaPartitionState::partitionIndex)
|
||||
.collect(Collectors.toList())))
|
||||
.collect(Collectors.toList());
|
||||
data.setTopics(topics);
|
||||
} else {
|
||||
data.setDeletePartitions(deletePartitions);
|
||||
List<StopReplicaPartitionV0> partitions = topicStates.stream().flatMap(topic ->
|
||||
topic.partitionStates().stream().map(partition ->
|
||||
new StopReplicaPartitionV0()
|
||||
.setTopicName(topic.topicName())
|
||||
.setPartitionIndex(partition.partitionIndex())))
|
||||
.collect(Collectors.toList());
|
||||
data.setUngroupedPartitions(partitions);
|
||||
}
|
||||
|
||||
return new StopReplicaRequest(data, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "(type=StopReplicaRequest" +
|
||||
", controllerId=" + controllerId +
|
||||
", controllerEpoch=" + controllerEpoch +
|
||||
", brokerEpoch=" + brokerEpoch +
|
||||
", deletePartitions=" + deletePartitions +
|
||||
", topicStates=" + topicStates.stream().map(StopReplicaTopicState::toString).collect(Collectors.joining(",")) +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
||||
private final StopReplicaRequestData data;
|
||||
|
||||
private StopReplicaRequest(StopReplicaRequestData data, short version) {
|
||||
super(ApiKeys.STOP_REPLICA, version);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopReplicaResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
Errors error = Errors.forException(e);
|
||||
|
||||
StopReplicaResponseData data = new StopReplicaResponseData();
|
||||
data.setErrorCode(error.code());
|
||||
|
||||
List<StopReplicaPartitionError> partitions = new ArrayList<>();
|
||||
for (StopReplicaTopicState topic : topicStates()) {
|
||||
for (StopReplicaPartitionState partition : topic.partitionStates()) {
|
||||
partitions.add(new StopReplicaPartitionError()
|
||||
.setTopicName(topic.topicName())
|
||||
.setPartitionIndex(partition.partitionIndex())
|
||||
.setErrorCode(error.code()));
|
||||
}
|
||||
}
|
||||
data.setPartitionErrors(partitions);
|
||||
|
||||
return new StopReplicaResponse(data);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<StopReplicaTopicState> topicStates() {
|
||||
if (version() < 1) {
|
||||
Map<String, StopReplicaTopicState> topicStates = new HashMap<>();
|
||||
for (StopReplicaPartitionV0 partition : data.ungroupedPartitions()) {
|
||||
StopReplicaTopicState topicState = topicStates.computeIfAbsent(partition.topicName(),
|
||||
topic -> new StopReplicaTopicState().setTopicName(topic));
|
||||
topicState.partitionStates().add(new StopReplicaPartitionState()
|
||||
.setPartitionIndex(partition.partitionIndex())
|
||||
.setDeletePartition(data.deletePartitions()));
|
||||
}
|
||||
return topicStates.values();
|
||||
} else if (version() < 3) {
|
||||
return () -> new MappedIterator<>(data.topics().iterator(), topic ->
|
||||
new StopReplicaTopicState()
|
||||
.setTopicName(topic.name())
|
||||
.setPartitionStates(topic.partitionIndexes().stream()
|
||||
.map(partition -> new StopReplicaPartitionState()
|
||||
.setPartitionIndex(partition)
|
||||
.setDeletePartition(data.deletePartitions()))
|
||||
.collect(Collectors.toList())));
|
||||
} else {
|
||||
return data.topicStates();
|
||||
}
|
||||
}
|
||||
|
||||
public Map<TopicPartition, StopReplicaPartitionState> partitionStates() {
|
||||
Map<TopicPartition, StopReplicaPartitionState> partitionStates = new HashMap<>();
|
||||
|
||||
if (version() < 1) {
|
||||
for (StopReplicaPartitionV0 partition : data.ungroupedPartitions()) {
|
||||
partitionStates.put(
|
||||
new TopicPartition(partition.topicName(), partition.partitionIndex()),
|
||||
new StopReplicaPartitionState()
|
||||
.setPartitionIndex(partition.partitionIndex())
|
||||
.setDeletePartition(data.deletePartitions()));
|
||||
}
|
||||
} else if (version() < 3) {
|
||||
for (StopReplicaTopicV1 topic : data.topics()) {
|
||||
for (Integer partitionIndex : topic.partitionIndexes()) {
|
||||
partitionStates.put(
|
||||
new TopicPartition(topic.name(), partitionIndex),
|
||||
new StopReplicaPartitionState()
|
||||
.setPartitionIndex(partitionIndex)
|
||||
.setDeletePartition(data.deletePartitions()));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (StopReplicaTopicState topicState : data.topicStates()) {
|
||||
for (StopReplicaPartitionState partitionState: topicState.partitionStates()) {
|
||||
partitionStates.put(
|
||||
new TopicPartition(topicState.topicName(), partitionState.partitionIndex()),
|
||||
partitionState);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return partitionStates;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int controllerId() {
|
||||
return data.controllerId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isKRaftController() {
|
||||
return data.isKRaftController();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int controllerEpoch() {
|
||||
return data.controllerEpoch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long brokerEpoch() {
|
||||
return data.brokerEpoch();
|
||||
}
|
||||
|
||||
public static StopReplicaRequest parse(ByteBuffer buffer, short version) {
|
||||
return new StopReplicaRequest(new StopReplicaRequestData(new ByteBufferAccessor(buffer), version), version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopReplicaRequestData data() {
|
||||
return data;
|
||||
}
|
||||
}
|
|
@ -1,88 +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.message.StopReplicaResponseData;
|
||||
import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class StopReplicaResponse extends AbstractResponse {
|
||||
|
||||
/**
|
||||
* Possible error code:
|
||||
* - {@link Errors#STALE_CONTROLLER_EPOCH}
|
||||
* - {@link Errors#STALE_BROKER_EPOCH}
|
||||
* - {@link Errors#FENCED_LEADER_EPOCH}
|
||||
* - {@link Errors#KAFKA_STORAGE_ERROR}
|
||||
*/
|
||||
private final StopReplicaResponseData data;
|
||||
|
||||
public StopReplicaResponse(StopReplicaResponseData data) {
|
||||
super(ApiKeys.STOP_REPLICA);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
public List<StopReplicaPartitionError> partitionErrors() {
|
||||
return data.partitionErrors();
|
||||
}
|
||||
|
||||
public Errors error() {
|
||||
return Errors.forCode(data.errorCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
if (data.errorCode() != Errors.NONE.code())
|
||||
// Minor optimization since the top-level error applies to all partitions
|
||||
return Collections.singletonMap(error(), data.partitionErrors().size() + 1);
|
||||
Map<Errors, Integer> errors = errorCounts(data.partitionErrors().stream().map(p -> Errors.forCode(p.errorCode())));
|
||||
updateErrorCounts(errors, Errors.forCode(data.errorCode())); // top level error
|
||||
return errors;
|
||||
}
|
||||
|
||||
public static StopReplicaResponse parse(ByteBuffer buffer, short version) {
|
||||
return new StopReplicaResponse(new StopReplicaResponseData(new ByteBufferAccessor(buffer), version));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int throttleTimeMs() {
|
||||
return DEFAULT_THROTTLE_TIME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maybeSetThrottleTimeMs(int throttleTimeMs) {
|
||||
// Not supported by the response schema
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopReplicaResponseData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return data.toString();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,255 +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.Uuid;
|
||||
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.security.auth.SecurityProtocol;
|
||||
import org.apache.kafka.common.utils.FlattenedIterator;
|
||||
|
||||
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 java.util.stream.Collectors;
|
||||
|
||||
import static java.util.Collections.singletonList;
|
||||
|
||||
public final class UpdateMetadataRequest extends AbstractControlRequest {
|
||||
|
||||
public static class Builder extends AbstractControlRequest.Builder<UpdateMetadataRequest> {
|
||||
private final List<UpdateMetadataPartitionState> partitionStates;
|
||||
private final List<UpdateMetadataBroker> liveBrokers;
|
||||
private final Map<String, Uuid> topicIds;
|
||||
private final Type updateType;
|
||||
|
||||
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
List<UpdateMetadataPartitionState> partitionStates, List<UpdateMetadataBroker> liveBrokers,
|
||||
Map<String, Uuid> topicIds) {
|
||||
this(version, controllerId, controllerEpoch, brokerEpoch, partitionStates,
|
||||
liveBrokers, topicIds, false, Type.UNKNOWN);
|
||||
}
|
||||
|
||||
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
|
||||
List<UpdateMetadataPartitionState> partitionStates, List<UpdateMetadataBroker> liveBrokers,
|
||||
Map<String, Uuid> topicIds, boolean kraftController, Type updateType) {
|
||||
super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch, kraftController);
|
||||
this.partitionStates = partitionStates;
|
||||
this.liveBrokers = liveBrokers;
|
||||
this.topicIds = topicIds;
|
||||
|
||||
if (version >= 8) {
|
||||
this.updateType = updateType;
|
||||
} else {
|
||||
this.updateType = Type.UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public UpdateMetadataRequest build(short version) {
|
||||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
UpdateMetadataRequestData data = new UpdateMetadataRequestData()
|
||||
.setControllerId(controllerId)
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setBrokerEpoch(brokerEpoch)
|
||||
.setLiveBrokers(liveBrokers);
|
||||
|
||||
if (version >= 8) {
|
||||
data.setIsKRaftController(kraftController);
|
||||
data.setType(updateType.toByte());
|
||||
}
|
||||
|
||||
if (version >= 5) {
|
||||
Map<String, UpdateMetadataTopicState> topicStatesMap = groupByTopic(topicIds, partitionStates);
|
||||
data.setTopicStates(new ArrayList<>(topicStatesMap.values()));
|
||||
} else {
|
||||
data.setUngroupedPartitionStates(partitionStates);
|
||||
}
|
||||
|
||||
return new UpdateMetadataRequest(data, version);
|
||||
}
|
||||
|
||||
private static Map<String, UpdateMetadataTopicState> groupByTopic(Map<String, Uuid> topicIds, List<UpdateMetadataPartitionState> partitionStates) {
|
||||
Map<String, UpdateMetadataTopicState> topicStates = new HashMap<>();
|
||||
for (UpdateMetadataPartitionState partition : partitionStates) {
|
||||
// We don't null out the topic name in UpdateMetadataPartitionState since it's ignored by the generated
|
||||
// code if version >= 5
|
||||
UpdateMetadataTopicState topicState = topicStates.computeIfAbsent(partition.topicName(),
|
||||
t -> new UpdateMetadataTopicState()
|
||||
.setTopicName(partition.topicName())
|
||||
.setTopicId(topicIds.getOrDefault(partition.topicName(), Uuid.ZERO_UUID))
|
||||
|
||||
);
|
||||
topicState.partitionStates().add(partition);
|
||||
}
|
||||
return topicStates;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "(type: UpdateMetadataRequest=" +
|
||||
", controllerId=" + controllerId +
|
||||
", controllerEpoch=" + controllerEpoch +
|
||||
", kraftController=" + kraftController +
|
||||
", type=" + updateType +
|
||||
", brokerEpoch=" + brokerEpoch +
|
||||
", partitionStates=" + partitionStates +
|
||||
", liveBrokers=" + liveBrokers.stream().map(UpdateMetadataBroker::toString).collect(Collectors.joining(", ")) +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
||||
private final UpdateMetadataRequestData data;
|
||||
|
||||
public 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();
|
||||
}
|
||||
|
||||
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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static String listenerNameFromSecurityProtocol(UpdateMetadataEndpoint endpoint) {
|
||||
SecurityProtocol securityProtocol = SecurityProtocol.forId(endpoint.securityProtocol());
|
||||
return ListenerName.forSecurityProtocol(securityProtocol).value();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int controllerId() {
|
||||
return data.controllerId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isKRaftController() {
|
||||
return data.isKRaftController();
|
||||
}
|
||||
|
||||
public Type updateType() {
|
||||
return Type.fromByte(data.type());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int controllerEpoch() {
|
||||
return data.controllerEpoch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long brokerEpoch() {
|
||||
return data.brokerEpoch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public UpdateMetadataResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
UpdateMetadataResponseData data = new UpdateMetadataResponseData()
|
||||
.setErrorCode(Errors.forException(e).code());
|
||||
return new UpdateMetadataResponse(data);
|
||||
}
|
||||
|
||||
public Iterable<UpdateMetadataPartitionState> partitionStates() {
|
||||
if (version() >= 5) {
|
||||
return () -> new FlattenedIterator<>(data.topicStates().iterator(),
|
||||
topicState -> topicState.partitionStates().iterator());
|
||||
}
|
||||
return data.ungroupedPartitionStates();
|
||||
}
|
||||
|
||||
public List<UpdateMetadataTopicState> topicStates() {
|
||||
if (version() >= 5) {
|
||||
return data.topicStates();
|
||||
}
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
public List<UpdateMetadataBroker> liveBrokers() {
|
||||
return data.liveBrokers();
|
||||
}
|
||||
|
||||
@Override
|
||||
public UpdateMetadataRequestData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
public static UpdateMetadataRequest parse(ByteBuffer buffer, short version) {
|
||||
return new UpdateMetadataRequest(new UpdateMetadataRequestData(new ByteBufferAccessor(buffer), version), version);
|
||||
}
|
||||
}
|
|
@ -1,64 +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.message.UpdateMetadataResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
|
||||
public class UpdateMetadataResponse extends AbstractResponse {
|
||||
|
||||
private final UpdateMetadataResponseData data;
|
||||
|
||||
public UpdateMetadataResponse(UpdateMetadataResponseData data) {
|
||||
super(ApiKeys.UPDATE_METADATA);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
public Errors error() {
|
||||
return Errors.forCode(data.errorCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
return errorCounts(error());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int throttleTimeMs() {
|
||||
return DEFAULT_THROTTLE_TIME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maybeSetThrottleTimeMs(int throttleTimeMs) {
|
||||
// Not supported by the response schema
|
||||
}
|
||||
|
||||
public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) {
|
||||
return new UpdateMetadataResponse(new UpdateMetadataResponseData(new ByteBufferAccessor(buffer), version));
|
||||
}
|
||||
|
||||
@Override
|
||||
public UpdateMetadataResponseData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
}
|
|
@ -54,7 +54,7 @@
|
|||
{ "name": "LeaderRecoveryState", "type": "int8", "versions": "1+", "default": "0",
|
||||
"about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." },
|
||||
{ "name": "PartitionEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The expected epoch of the partition which is being updated. For legacy cluster this is the ZkVersion in the LeaderAndIsr request." }
|
||||
"about": "The expected epoch of the partition which is being updated." }
|
||||
]}
|
||||
]}
|
||||
]
|
||||
|
|
|
@ -51,7 +51,7 @@
|
|||
{ "name": "LeaderRecoveryState", "type": "int8", "versions": "1+", "default": "0", "ignorable": true,
|
||||
"about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." },
|
||||
{ "name": "PartitionEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers." }
|
||||
"about": "The current epoch for the partition for KRaft controllers." }
|
||||
]}
|
||||
]}
|
||||
]
|
||||
|
|
|
@ -16,21 +16,7 @@
|
|||
{
|
||||
"apiKey": 7,
|
||||
"type": "request",
|
||||
"listeners": ["zkBroker", "controller"],
|
||||
"name": "ControlledShutdownRequest",
|
||||
// Version 0 of ControlledShutdownRequest has a non-standard request header
|
||||
// which does not include clientId. Version 1 and later use the standard
|
||||
// request header.
|
||||
//
|
||||
// Version 1 is the same as version 0.
|
||||
//
|
||||
// Version 2 adds BrokerEpoch.
|
||||
"validVersions": "0-3",
|
||||
"flexibleVersions": "3+",
|
||||
"fields": [
|
||||
{ "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The id of the broker for which controlled shutdown has been requested." },
|
||||
{ "name": "BrokerEpoch", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true,
|
||||
"about": "The broker epoch." }
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -17,18 +17,6 @@
|
|||
"apiKey": 7,
|
||||
"type": "response",
|
||||
"name": "ControlledShutdownResponse",
|
||||
// Versions 1 and 2 are the same as version 0.
|
||||
"validVersions": "0-3",
|
||||
"flexibleVersions": "3+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The top-level error code." },
|
||||
{ "name": "RemainingPartitions", "type": "[]RemainingPartition", "versions": "0+",
|
||||
"about": "The partitions that the broker still leads.", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName",
|
||||
"about": "The name of the topic." },
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true,
|
||||
"about": "The index of the partition." }
|
||||
]}
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -16,84 +16,7 @@
|
|||
{
|
||||
"apiKey": 4,
|
||||
"type": "request",
|
||||
"listeners": ["zkBroker"],
|
||||
"name": "LeaderAndIsrRequest",
|
||||
// Version 1 adds IsNew.
|
||||
//
|
||||
// Version 2 adds broker epoch and reorganizes the partitions by topic.
|
||||
//
|
||||
// Version 3 adds AddingReplicas and RemovingReplicas.
|
||||
//
|
||||
// Version 4 is the first flexible version.
|
||||
//
|
||||
// Version 5 adds Topic ID and Type to the TopicStates, as described in KIP-516.
|
||||
//
|
||||
// Version 6 adds LeaderRecoveryState as described in KIP-704.
|
||||
//
|
||||
// Version 7 adds KRaft Controller ID field as part of KIP-866
|
||||
"validVersions": "0-7",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The current controller ID." },
|
||||
{ "name": "isKRaftController", "type": "bool", "versions": "7+", "default": "false",
|
||||
"about": "If KRaft controller id is used during migration. See KIP-866." },
|
||||
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current controller epoch." },
|
||||
{ "name": "BrokerEpoch", "type": "int64", "versions": "2+", "ignorable": true, "default": "-1",
|
||||
"about": "The current broker epoch." },
|
||||
{ "name": "Type", "type": "int8", "versions": "5+",
|
||||
"about": "The type that indicates whether all topics are included in the request."},
|
||||
{ "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": "[]LeaderAndIsrTopicState", "versions": "2+",
|
||||
"about": "Each topic.", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "2+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "TopicId", "type": "uuid", "versions": "5+", "ignorable": true,
|
||||
"about": "The unique topic ID." },
|
||||
{ "name": "PartitionStates", "type": "[]LeaderAndIsrPartitionState", "versions": "2+",
|
||||
"about": "The state of each partition." }
|
||||
]},
|
||||
{ "name": "LiveLeaders", "type": "[]LeaderAndIsrLiveLeader", "versions": "0+",
|
||||
"about": "The current live leaders.", "fields": [
|
||||
{ "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The leader's broker ID." },
|
||||
{ "name": "HostName", "type": "string", "versions": "0+",
|
||||
"about": "The leader's hostname." },
|
||||
{ "name": "Port", "type": "int32", "versions": "0+",
|
||||
"about": "The leader's port." }
|
||||
]}
|
||||
],
|
||||
"commonStructs": [
|
||||
{ "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": "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": "Isr", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The in-sync replica IDs." },
|
||||
{ "name": "PartitionEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion)." },
|
||||
{ "name": "Replicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The replica IDs." },
|
||||
{ "name": "AddingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true, "entityType": "brokerId",
|
||||
"about": "The replica IDs that we are adding this partition to, or null if no replicas are being added." },
|
||||
{ "name": "RemovingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true, "entityType": "brokerId",
|
||||
"about": "The replica IDs that we are removing this partition from, or null if no replicas are being removed." },
|
||||
{ "name": "IsNew", "type": "bool", "versions": "1+", "default": "false", "ignorable": true,
|
||||
"about": "Whether the replica should have existed on the broker or not." },
|
||||
{ "name": "LeaderRecoveryState", "type": "int8", "versions": "6+", "default": "0",
|
||||
"about": "1 if the partition is recovering from an unclean leader election; 0 otherwise." }
|
||||
]}
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -17,39 +17,6 @@
|
|||
"apiKey": 4,
|
||||
"type": "response",
|
||||
"name": "LeaderAndIsrResponse",
|
||||
// Version 1 adds KAFKA_STORAGE_ERROR as a valid error code.
|
||||
//
|
||||
// Version 2 is the same as version 1.
|
||||
//
|
||||
// Version 3 is the same as version 2.
|
||||
//
|
||||
// Version 4 is the first flexible version.
|
||||
//
|
||||
// Version 5 removes TopicName and replaces it with TopicId and reorganizes
|
||||
// the partitions by topic, as described by KIP-516.
|
||||
"validVersions": "0-7",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The error code, or 0 if there was no error." },
|
||||
{ "name": "PartitionErrors", "type": "[]LeaderAndIsrPartitionError", "versions": "0-4",
|
||||
"about": "Each partition in v0 to v4 message."},
|
||||
{ "name": "Topics", "type": "[]LeaderAndIsrTopicError", "versions": "5+",
|
||||
"about": "Each topic.", "fields": [
|
||||
{ "name": "TopicId", "type": "uuid", "versions": "5+", "mapKey": true,
|
||||
"about": "The unique topic ID." },
|
||||
{ "name": "PartitionErrors", "type": "[]LeaderAndIsrPartitionError", "versions": "5+",
|
||||
"about": "Each partition."}
|
||||
]}
|
||||
],
|
||||
"commonStructs": [
|
||||
{ "name": "LeaderAndIsrPartitionError", "versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0-4", "entityType": "topicName", "ignorable": true,
|
||||
"about": "The topic name."},
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The partition error code, or 0 if there was no error." }
|
||||
]}
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -16,12 +16,14 @@
|
|||
{
|
||||
"type": "header",
|
||||
"name": "RequestHeader",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
//
|
||||
// Version 0 of the RequestHeader is only used by v0 of ControlledShutdownRequest.
|
||||
//
|
||||
// Version 1 is the first version with ClientId.
|
||||
//
|
||||
// Version 2 is the first flexible version.
|
||||
"validVersions": "0-2",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "RequestApiKey", "type": "int16", "versions": "0+",
|
||||
|
@ -36,7 +38,7 @@
|
|||
// ApiVersionsRequest, even if it is from a newer version.
|
||||
// Since the client is sending the ApiVersionsRequest in order to discover what
|
||||
// versions are supported, the client does not know the best version to use.
|
||||
{ "name": "ClientId", "type": "string", "versions": "1+", "nullableVersions": "1+", "ignorable": true,
|
||||
"flexibleVersions": "none", "about": "The client ID string." }
|
||||
{ "name": "ClientId", "type": "string", "versions": "1+", "nullableVersions": "1+", "flexibleVersions": "none",
|
||||
"about": "The client ID string." }
|
||||
]
|
||||
}
|
||||
|
|
|
@ -16,56 +16,7 @@
|
|||
{
|
||||
"apiKey": 5,
|
||||
"type": "request",
|
||||
"listeners": ["zkBroker"],
|
||||
"name": "StopReplicaRequest",
|
||||
// Version 1 adds the broker epoch and reorganizes the partitions to be stored
|
||||
// per topic.
|
||||
//
|
||||
// Version 2 is the first flexible version.
|
||||
//
|
||||
// Version 3 adds the leader epoch per partition (KIP-570).
|
||||
//
|
||||
// Version 4 adds KRaft Controller ID field as part of KIP-866
|
||||
"validVersions": "0-4",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The controller id." },
|
||||
{ "name": "isKRaftController", "type": "bool", "versions": "4+", "default": "false",
|
||||
"about": "If KRaft controller id is used during migration. See KIP-866." },
|
||||
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The controller epoch." },
|
||||
{ "name": "BrokerEpoch", "type": "int64", "versions": "1+", "default": "-1", "ignorable": true,
|
||||
"about": "The broker epoch." },
|
||||
{ "name": "DeletePartitions", "type": "bool", "versions": "0-2",
|
||||
"about": "Whether these partitions should be deleted." },
|
||||
{ "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": "[]StopReplicaTopicV1", "versions": "1-2",
|
||||
"about": "The topics to stop.", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "1-2", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "PartitionIndexes", "type": "[]int32", "versions": "1-2",
|
||||
"about": "The partition indexes." }
|
||||
]},
|
||||
{ "name": "TopicStates", "type": "[]StopReplicaTopicState", "versions": "3+",
|
||||
"about": "Each topic.", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "3+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "PartitionStates", "type": "[]StopReplicaPartitionState", "versions": "3+",
|
||||
"about": "The state of each partition.", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "3+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "LeaderEpoch", "type": "int32", "versions": "3+", "default": "-1",
|
||||
"about": "The leader epoch." },
|
||||
{ "name": "DeletePartition", "type": "bool", "versions": "3+",
|
||||
"about": "Whether this partition should be deleted." }
|
||||
]}
|
||||
]}
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -17,24 +17,6 @@
|
|||
"apiKey": 5,
|
||||
"type": "response",
|
||||
"name": "StopReplicaResponse",
|
||||
// Version 1 is the same as version 0.
|
||||
//
|
||||
// Version 2 is the first flexible version.
|
||||
//
|
||||
// Version 3 returns FENCED_LEADER_EPOCH if the epoch of the leader is stale (KIP-570).
|
||||
"validVersions": "0-4",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The top-level error code, or 0 if there was no top-level error." },
|
||||
{ "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." },
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The partition error code, or 0 if there was no partition error." }
|
||||
]}
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -16,90 +16,7 @@
|
|||
{
|
||||
"apiKey": 6,
|
||||
"type": "request",
|
||||
"listeners": ["zkBroker"],
|
||||
"name": "UpdateMetadataRequest",
|
||||
// Version 1 allows specifying multiple endpoints for each broker.
|
||||
//
|
||||
// Version 2 adds the rack.
|
||||
//
|
||||
// Version 3 adds the listener name.
|
||||
//
|
||||
// Version 4 adds the offline replica list.
|
||||
//
|
||||
// Version 5 adds the broker epoch field and normalizes partitions by topic.
|
||||
//
|
||||
// Version 7 adds topicId
|
||||
//
|
||||
// Version 8 adds KRaft Controller ID field as part of KIP-866
|
||||
"validVersions": "0-8",
|
||||
"flexibleVersions": "6+",
|
||||
"fields": [
|
||||
{ "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The controller id." },
|
||||
{ "name": "isKRaftController", "type": "bool", "versions": "8+", "default": "false",
|
||||
"about": "If KRaft controller id is used during migration. See KIP-866." },
|
||||
{ "name": "Type", "type": "int8", "versions": "8+",
|
||||
"default": 0, "tag": 0, "taggedVersions": "8+",
|
||||
"about": "Indicates if this request is a Full metadata snapshot (2), Incremental (1), or Unknown (0). Using during ZK migration, see KIP-866."},
|
||||
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The controller epoch." },
|
||||
{ "name": "BrokerEpoch", "type": "int64", "versions": "5+", "ignorable": true, "default": "-1",
|
||||
"about": "The broker epoch." },
|
||||
{ "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": "[]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": "TopicId", "type": "uuid", "versions": "7+", "ignorable": true, "about": "The topic id."},
|
||||
{ "name": "PartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "5+",
|
||||
"about": "The partition that we would like to update." }
|
||||
]},
|
||||
{ "name": "LiveBrokers", "type": "[]UpdateMetadataBroker", "versions": "0+",
|
||||
"about": "The brokers that we know about.", "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
|
||||
// port per broker, rather than an array of endpoints.
|
||||
{ "name": "V0Host", "type": "string", "versions": "0", "ignorable": true,
|
||||
"about": "The broker hostname." },
|
||||
{ "name": "V0Port", "type": "int32", "versions": "0", "ignorable": true,
|
||||
"about": "The broker port." },
|
||||
{ "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+", "ignorable": true,
|
||||
"about": "The listener name." },
|
||||
{ "name": "SecurityProtocol", "type": "int16", "versions": "1+",
|
||||
"about": "The security protocol type." }
|
||||
]},
|
||||
{ "name": "Rack", "type": "string", "versions": "2+", "nullableVersions": "0+", "ignorable": true,
|
||||
"about": "The rack which this broker belongs to." }
|
||||
]}
|
||||
],
|
||||
"commonStructs": [
|
||||
{ "name": "UpdateMetadataPartitionState", "versions": "0+", "fields": [
|
||||
{ "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." },
|
||||
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The controller epoch." },
|
||||
{ "name": "Leader", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The ID of the broker which is the current partition leader." },
|
||||
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The leader epoch of this partition." },
|
||||
{ "name": "Isr", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The brokers which are in the ISR for this partition." },
|
||||
{ "name": "ZkVersion", "type": "int32", "versions": "0+",
|
||||
"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", "ignorable": true,
|
||||
"about": "The replicas of this partition which are offline." }
|
||||
]}
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -17,11 +17,6 @@
|
|||
"apiKey": 6,
|
||||
"type": "response",
|
||||
"name": "UpdateMetadataResponse",
|
||||
// Versions 1, 2, 3, 4, and 5 are the same as version 0
|
||||
"validVersions": "0-8",
|
||||
"flexibleVersions": "6+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The error code, or 0 if there was no error." }
|
||||
]
|
||||
// This request was removed in Apache Kafka 4.0.
|
||||
"validVersions": "none"
|
||||
}
|
||||
|
|
|
@ -18,8 +18,10 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"name": "WriteTxnMarkersRequest",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
//
|
||||
// Version 1 enables flexible versions.
|
||||
"validVersions": "0-1",
|
||||
"validVersions": "1",
|
||||
"flexibleVersions": "1+",
|
||||
"fields": [
|
||||
{ "name": "Markers", "type": "[]WritableTxnMarker", "versions": "0+",
|
||||
|
|
|
@ -17,7 +17,9 @@
|
|||
"apiKey": 27,
|
||||
"type": "response",
|
||||
"name": "WriteTxnMarkersResponse",
|
||||
"validVersions": "0-1",
|
||||
"validVersions": "1",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
//
|
||||
// Version 1 enables flexible versions.
|
||||
"flexibleVersions": "1+",
|
||||
"fields": [
|
||||
|
|
|
@ -80,6 +80,8 @@ public class NodeApiVersionsTest {
|
|||
bld.append(prefix);
|
||||
if (apiKey == ApiKeys.DELETE_TOPICS) {
|
||||
bld.append("DeleteTopics(20): 10000 to 10001 [unusable: node too new]");
|
||||
} else if (!apiKey.hasValidVersion()) {
|
||||
bld.append(apiKey.name + "(" + apiKey.id + "): 0 to -1 [unusable: node too new]");
|
||||
} else {
|
||||
bld.append(apiKey.name).append("(").
|
||||
append(apiKey.id).append("): ");
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.kafka.common.message;
|
||||
|
||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.types.Schema;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -30,6 +31,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
|
@ -59,22 +61,28 @@ public class ApiMessageTypeTest {
|
|||
Set<Short> ids = new HashSet<>();
|
||||
Set<String> requestNames = new HashSet<>();
|
||||
Set<String> responseNames = new HashSet<>();
|
||||
int apiKeysWithNoValidVersionCount = 0;
|
||||
for (ApiMessageType type : ApiMessageType.values()) {
|
||||
assertFalse(ids.contains(type.apiKey()),
|
||||
"found two ApiMessageType objects with id " + type.apiKey());
|
||||
ids.add(type.apiKey());
|
||||
String requestName = type.newRequest().getClass().getSimpleName();
|
||||
assertFalse(requestNames.contains(requestName),
|
||||
"found two ApiMessageType objects with requestName " + requestName);
|
||||
requestNames.add(requestName);
|
||||
String responseName = type.newResponse().getClass().getSimpleName();
|
||||
assertFalse(responseNames.contains(responseName),
|
||||
"found two ApiMessageType objects with responseName " + responseName);
|
||||
responseNames.add(responseName);
|
||||
ApiKeys apiKey = ApiKeys.forId(type.apiKey());
|
||||
if (apiKey.hasValidVersion()) {
|
||||
String requestName = type.newRequest().getClass().getSimpleName();
|
||||
assertFalse(requestNames.contains(requestName),
|
||||
"found two ApiMessageType objects with requestName " + requestName);
|
||||
requestNames.add(requestName);
|
||||
String responseName = type.newResponse().getClass().getSimpleName();
|
||||
assertFalse(responseNames.contains(responseName),
|
||||
"found two ApiMessageType objects with responseName " + responseName);
|
||||
responseNames.add(responseName);
|
||||
} else
|
||||
++apiKeysWithNoValidVersionCount;
|
||||
}
|
||||
assertEquals(ApiMessageType.values().length, ids.size());
|
||||
assertEquals(ApiMessageType.values().length, requestNames.size());
|
||||
assertEquals(ApiMessageType.values().length, responseNames.size());
|
||||
int expectedNamesCount = ApiMessageType.values().length - apiKeysWithNoValidVersionCount;
|
||||
assertEquals(expectedNamesCount, requestNames.size());
|
||||
assertEquals(expectedNamesCount, responseNames.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -85,12 +93,6 @@ public class ApiMessageTypeTest {
|
|||
assertEquals((short) 1, ApiMessageType.PRODUCE.requestHeaderVersion((short) 1));
|
||||
assertEquals((short) 0, ApiMessageType.PRODUCE.responseHeaderVersion((short) 1));
|
||||
|
||||
assertEquals((short) 0, ApiMessageType.CONTROLLED_SHUTDOWN.requestHeaderVersion((short) 0));
|
||||
assertEquals((short) 0, ApiMessageType.CONTROLLED_SHUTDOWN.responseHeaderVersion((short) 0));
|
||||
|
||||
assertEquals((short) 1, ApiMessageType.CONTROLLED_SHUTDOWN.requestHeaderVersion((short) 1));
|
||||
assertEquals((short) 0, ApiMessageType.CONTROLLED_SHUTDOWN.responseHeaderVersion((short) 1));
|
||||
|
||||
assertEquals((short) 1, ApiMessageType.CREATE_TOPICS.requestHeaderVersion((short) 4));
|
||||
assertEquals((short) 0, ApiMessageType.CREATE_TOPICS.responseHeaderVersion((short) 4));
|
||||
|
||||
|
@ -98,6 +100,16 @@ public class ApiMessageTypeTest {
|
|||
assertEquals((short) 1, ApiMessageType.CREATE_TOPICS.responseHeaderVersion((short) 5));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHeaderVersionWithNoValidVersion() {
|
||||
for (ApiMessageType messageType : ApiMessageType.values()) {
|
||||
if (messageType.lowestSupportedVersion() > messageType.highestSupportedVersion(true)) {
|
||||
assertThrows(UnsupportedVersionException.class, () -> messageType.requestHeaderVersion((short) 0));
|
||||
assertThrows(UnsupportedVersionException.class, () -> messageType.responseHeaderVersion((short) 0));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAllVersionsHaveSchemas() {
|
||||
for (ApiMessageType type : ApiMessageType.values()) {
|
||||
|
|
|
@ -409,35 +409,6 @@ public final class MessageTest {
|
|||
new OffsetForLeaderEpochRequestData().setReplicaId(-2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeaderAndIsrVersions() throws Exception {
|
||||
// Version 3 adds two new fields - AddingReplicas and RemovingReplicas
|
||||
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)));
|
||||
testAllMessageRoundTripsFromVersion((short) 3, new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOffsetCommitRequestVersions() throws Exception {
|
||||
String groupId = "groupId";
|
||||
|
@ -1095,23 +1066,25 @@ public final class MessageTest {
|
|||
@Test
|
||||
public void testMessageVersions() {
|
||||
for (ApiKeys apiKey : ApiKeys.values()) {
|
||||
Message message = null;
|
||||
try {
|
||||
message = ApiMessageType.fromApiKey(apiKey.id).newRequest();
|
||||
} catch (UnsupportedVersionException e) {
|
||||
fail("No request message spec found for API " + apiKey);
|
||||
if (apiKey.hasValidVersion()) {
|
||||
Message message = null;
|
||||
try {
|
||||
message = ApiMessageType.fromApiKey(apiKey.id).newRequest();
|
||||
} catch (UnsupportedVersionException e) {
|
||||
fail("No request message spec found for API " + apiKey);
|
||||
}
|
||||
assertTrue(apiKey.latestVersion() <= message.highestSupportedVersion(),
|
||||
"Request message spec for " + apiKey + " only " + "supports versions up to " +
|
||||
message.highestSupportedVersion());
|
||||
try {
|
||||
message = ApiMessageType.fromApiKey(apiKey.id).newResponse();
|
||||
} catch (UnsupportedVersionException e) {
|
||||
fail("No response message spec found for API " + apiKey);
|
||||
}
|
||||
assertTrue(apiKey.latestVersion() <= message.highestSupportedVersion(),
|
||||
"Response message spec for " + apiKey + " only " + "supports versions up to " +
|
||||
message.highestSupportedVersion());
|
||||
}
|
||||
assertTrue(apiKey.latestVersion() <= message.highestSupportedVersion(),
|
||||
"Request message spec for " + apiKey + " only " + "supports versions up to " +
|
||||
message.highestSupportedVersion());
|
||||
try {
|
||||
message = ApiMessageType.fromApiKey(apiKey.id).newResponse();
|
||||
} catch (UnsupportedVersionException e) {
|
||||
fail("No response message spec found for API " + apiKey);
|
||||
}
|
||||
assertTrue(apiKey.latestVersion() <= message.highestSupportedVersion(),
|
||||
"Response message spec for " + apiKey + " only " + "supports versions up to " +
|
||||
message.highestSupportedVersion());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -80,7 +80,7 @@ public class ApiKeysTest {
|
|||
public void testApiScope() {
|
||||
Set<ApiKeys> apisMissingScope = new HashSet<>();
|
||||
for (ApiKeys apiKey : ApiKeys.values()) {
|
||||
if (apiKey.messageType.listeners().isEmpty()) {
|
||||
if (apiKey.messageType.listeners().isEmpty() && apiKey.hasValidVersion()) {
|
||||
apisMissingScope.add(apiKey);
|
||||
}
|
||||
}
|
||||
|
@ -88,6 +88,18 @@ public class ApiKeysTest {
|
|||
"Found some APIs missing scope definition");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHasValidVersions() {
|
||||
var apiKeysWithNoValidVersions = Set.of(ApiKeys.LEADER_AND_ISR, ApiKeys.STOP_REPLICA, ApiKeys.UPDATE_METADATA,
|
||||
ApiKeys.CONTROLLED_SHUTDOWN);
|
||||
for (ApiKeys apiKey : ApiKeys.values()) {
|
||||
if (apiKeysWithNoValidVersions.contains(apiKey))
|
||||
assertFalse(apiKey.hasValidVersion());
|
||||
else
|
||||
assertTrue(apiKey.hasValidVersion());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHtmlOnlyHaveStableApi() {
|
||||
String html = ApiKeys.toHtml();
|
||||
|
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* 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.protocol;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
||||
public class ProtocolTest {
|
||||
|
||||
@Test
|
||||
public void testToHtml() {
|
||||
var html = Protocol.toHtml();
|
||||
assertFalse(html.isBlank());
|
||||
assertFalse(html.contains("LeaderAndIsr"), "Removed LeaderAndIsr should not show in HTML");
|
||||
}
|
||||
|
||||
}
|
|
@ -1,52 +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.errors.ClusterAuthorizationException;
|
||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||
import org.apache.kafka.common.message.ControlledShutdownRequestData;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.CONTROLLED_SHUTDOWN;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
public class ControlledShutdownRequestTest {
|
||||
|
||||
@Test
|
||||
public void testUnsupportedVersion() {
|
||||
ControlledShutdownRequest.Builder builder = new ControlledShutdownRequest.Builder(
|
||||
new ControlledShutdownRequestData().setBrokerId(1),
|
||||
(short) (CONTROLLED_SHUTDOWN.latestVersion() + 1));
|
||||
assertThrows(UnsupportedVersionException.class, builder::build);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetErrorResponse() {
|
||||
for (short version : CONTROLLED_SHUTDOWN.allVersions()) {
|
||||
ControlledShutdownRequest.Builder builder = new ControlledShutdownRequest.Builder(
|
||||
new ControlledShutdownRequestData().setBrokerId(1), version);
|
||||
ControlledShutdownRequest request = builder.build();
|
||||
ControlledShutdownResponse response = request.getErrorResponse(0,
|
||||
new ClusterAuthorizationException("Not authorized"));
|
||||
assertEquals(Errors.CLUSTER_AUTHORIZATION_FAILED, response.error());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -32,32 +32,6 @@ import static org.mockito.Mockito.verify;
|
|||
|
||||
public class RequestHeaderTest {
|
||||
|
||||
@Test
|
||||
public void testSerdeControlledShutdownV0() {
|
||||
// Verify that version 0 of controlled shutdown does not include the clientId field
|
||||
short apiVersion = 0;
|
||||
int correlationId = 2342;
|
||||
ByteBuffer rawBuffer = ByteBuffer.allocate(32);
|
||||
rawBuffer.putShort(ApiKeys.CONTROLLED_SHUTDOWN.id);
|
||||
rawBuffer.putShort(apiVersion);
|
||||
rawBuffer.putInt(correlationId);
|
||||
rawBuffer.flip();
|
||||
|
||||
RequestHeader deserialized = RequestHeader.parse(rawBuffer);
|
||||
assertEquals(ApiKeys.CONTROLLED_SHUTDOWN, deserialized.apiKey());
|
||||
assertEquals(0, deserialized.apiVersion());
|
||||
assertEquals(correlationId, deserialized.correlationId());
|
||||
assertEquals("", deserialized.clientId());
|
||||
assertEquals(0, deserialized.headerVersion());
|
||||
|
||||
ByteBuffer serializedBuffer = RequestTestUtils.serializeRequestHeader(deserialized);
|
||||
|
||||
assertEquals(ApiKeys.CONTROLLED_SHUTDOWN.id, serializedBuffer.getShort(0));
|
||||
assertEquals(0, serializedBuffer.getShort(2));
|
||||
assertEquals(correlationId, serializedBuffer.getInt(4));
|
||||
assertEquals(8, serializedBuffer.limit());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequestHeaderV1() {
|
||||
short apiVersion = 1;
|
||||
|
|
|
@ -77,10 +77,6 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeRequestData;
|
|||
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
|
||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
|
||||
import org.apache.kafka.common.message.ControlledShutdownRequestData;
|
||||
import org.apache.kafka.common.message.ControlledShutdownResponseData;
|
||||
import org.apache.kafka.common.message.ControlledShutdownResponseData.RemainingPartition;
|
||||
import org.apache.kafka.common.message.ControlledShutdownResponseData.RemainingPartitionCollection;
|
||||
import org.apache.kafka.common.message.ControllerRegistrationRequestData;
|
||||
import org.apache.kafka.common.message.ControllerRegistrationResponseData;
|
||||
import org.apache.kafka.common.message.CreateAclsRequestData;
|
||||
|
@ -173,9 +169,6 @@ import org.apache.kafka.common.message.InitializeShareGroupStateResponseData;
|
|||
import org.apache.kafka.common.message.JoinGroupRequestData;
|
||||
import org.apache.kafka.common.message.JoinGroupResponseData;
|
||||
import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicErrorCollection;
|
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
|
||||
import org.apache.kafka.common.message.LeaveGroupResponseData;
|
||||
import org.apache.kafka.common.message.ListClientMetricsResourcesRequestData;
|
||||
|
@ -232,9 +225,6 @@ import org.apache.kafka.common.message.ShareGroupDescribeRequestData;
|
|||
import org.apache.kafka.common.message.ShareGroupDescribeResponseData;
|
||||
import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData;
|
||||
import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopicState;
|
||||
import org.apache.kafka.common.message.StopReplicaResponseData;
|
||||
import org.apache.kafka.common.message.StreamsGroupDescribeRequestData;
|
||||
import org.apache.kafka.common.message.StreamsGroupDescribeResponseData;
|
||||
import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData;
|
||||
|
@ -246,17 +236,12 @@ import org.apache.kafka.common.message.UnregisterBrokerRequestData;
|
|||
import org.apache.kafka.common.message.UnregisterBrokerResponseData;
|
||||
import org.apache.kafka.common.message.UpdateFeaturesRequestData;
|
||||
import org.apache.kafka.common.message.UpdateFeaturesResponseData;
|
||||
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.message.UpdateRaftVoterRequestData;
|
||||
import org.apache.kafka.common.message.UpdateRaftVoterResponseData;
|
||||
import org.apache.kafka.common.message.VoteRequestData;
|
||||
import org.apache.kafka.common.message.VoteResponseData;
|
||||
import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
|
||||
import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.network.Send;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
|
@ -302,14 +287,12 @@ import java.util.LinkedList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.API_VERSIONS;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.CONTROLLED_SHUTDOWN;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.CREATE_PARTITIONS;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.CREATE_TOPICS;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.DELETE_ACLS;
|
||||
|
@ -327,9 +310,7 @@ import static org.apache.kafka.common.protocol.ApiKeys.METADATA;
|
|||
import static org.apache.kafka.common.protocol.ApiKeys.OFFSET_FETCH;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.SASL_AUTHENTICATE;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.STOP_REPLICA;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.SYNC_GROUP;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.UPDATE_METADATA;
|
||||
import static org.apache.kafka.common.protocol.ApiKeys.WRITE_TXN_MARKERS;
|
||||
import static org.apache.kafka.common.requests.EndTxnRequest.LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2;
|
||||
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
|
||||
|
@ -397,20 +378,10 @@ public class RequestResponseTest {
|
|||
new NotCoordinatorException("Not Coordinator"));
|
||||
checkErrorResponse(createOffsetFetchRequestWithMultipleGroups((short) 8, true),
|
||||
new NotCoordinatorException("Not Coordinator"));
|
||||
// StopReplica
|
||||
for (short version : STOP_REPLICA.allVersions()) {
|
||||
checkRequest(createStopReplicaRequest(version, false));
|
||||
checkErrorResponse(createStopReplicaRequest(version, false), unknownServerException);
|
||||
}
|
||||
// CreatePartitions
|
||||
for (short version : CREATE_PARTITIONS.allVersions()) {
|
||||
checkRequest(createCreatePartitionsRequestWithAssignments(version));
|
||||
}
|
||||
// UpdateMetadata
|
||||
for (short version : UPDATE_METADATA.allVersions()) {
|
||||
checkRequest(createUpdateMetadataRequest(version, null));
|
||||
checkErrorResponse(createUpdateMetadataRequest(version, null), unknownServerException);
|
||||
}
|
||||
// LeaderForEpoch
|
||||
checkRequest(createLeaderEpochRequestForConsumer());
|
||||
checkErrorResponse(createLeaderEpochRequestForConsumer(), unknownServerException);
|
||||
|
@ -935,7 +906,6 @@ public class RequestResponseTest {
|
|||
assertEquals(1, createApiVersionResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createBrokerHeartbeatResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createBrokerRegistrationResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createControlledShutdownResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(2, createCreateAclsResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createCreatePartitionsResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createCreateTokenResponse().errorCounts().get(Errors.NONE));
|
||||
|
@ -958,8 +928,6 @@ public class RequestResponseTest {
|
|||
assertEquals(1, createHeartBeatResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createIncrementalAlterConfigsResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createJoinGroupResponse(JOIN_GROUP.latestVersion()).errorCounts().get(Errors.NONE));
|
||||
assertEquals(2, createLeaderAndIsrResponse((short) 4).errorCounts().get(Errors.NONE));
|
||||
assertEquals(2, createLeaderAndIsrResponse((short) 5).errorCounts().get(Errors.NONE));
|
||||
assertEquals(3, createLeaderEpochResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createLeaveGroupResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createListGroupsResponse(LIST_GROUPS.latestVersion()).errorCounts().get(Errors.NONE));
|
||||
|
@ -973,10 +941,8 @@ public class RequestResponseTest {
|
|||
assertEquals(1, createRenewTokenResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createSaslAuthenticateResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createSaslHandshakeResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(2, createStopReplicaResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createSyncGroupResponse(SYNC_GROUP.latestVersion()).errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createTxnOffsetCommitResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createUpdateMetadataResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createWriteTxnMarkersResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createShareGroupHeartbeatResponse().errorCounts().get(Errors.NONE));
|
||||
assertEquals(1, createShareGroupDescribeResponse().errorCounts().get(Errors.NONE));
|
||||
|
@ -990,10 +956,6 @@ public class RequestResponseTest {
|
|||
case FETCH: return createFetchRequest(version);
|
||||
case LIST_OFFSETS: return createListOffsetRequest(version);
|
||||
case METADATA: return createMetadataRequest(version, singletonList("topic1"));
|
||||
case LEADER_AND_ISR: return createLeaderAndIsrRequest(version);
|
||||
case STOP_REPLICA: return createStopReplicaRequest(version, true);
|
||||
case UPDATE_METADATA: return createUpdateMetadataRequest(version, "rack1");
|
||||
case CONTROLLED_SHUTDOWN: return createControlledShutdownRequest(version);
|
||||
case OFFSET_COMMIT: return createOffsetCommitRequest(version);
|
||||
case OFFSET_FETCH: return createOffsetFetchRequest(version, true);
|
||||
case FIND_COORDINATOR: return createFindCoordinatorRequest(version);
|
||||
|
@ -1087,10 +1049,6 @@ public class RequestResponseTest {
|
|||
case FETCH: return createFetchResponse(version);
|
||||
case LIST_OFFSETS: return createListOffsetResponse(version);
|
||||
case METADATA: return createMetadataResponse();
|
||||
case LEADER_AND_ISR: return createLeaderAndIsrResponse(version);
|
||||
case STOP_REPLICA: return createStopReplicaResponse();
|
||||
case UPDATE_METADATA: return createUpdateMetadataResponse();
|
||||
case CONTROLLED_SHUTDOWN: return createControlledShutdownResponse();
|
||||
case OFFSET_COMMIT: return createOffsetCommitResponse();
|
||||
case OFFSET_FETCH: return createOffsetFetchResponse(version);
|
||||
case FIND_COORDINATOR: return createFindCoordinatorResponse(version);
|
||||
|
@ -2533,224 +2491,6 @@ public class RequestResponseTest {
|
|||
return new ProduceResponse(responseData, 0);
|
||||
}
|
||||
|
||||
private StopReplicaRequest createStopReplicaRequest(short version, boolean deletePartitions) {
|
||||
List<StopReplicaTopicState> topicStates = new ArrayList<>();
|
||||
StopReplicaTopicState topic1 = new StopReplicaTopicState()
|
||||
.setTopicName("topic1")
|
||||
.setPartitionStates(singletonList(new StopReplicaPartitionState()
|
||||
.setPartitionIndex(0)
|
||||
.setLeaderEpoch(1)
|
||||
.setDeletePartition(deletePartitions)));
|
||||
topicStates.add(topic1);
|
||||
StopReplicaTopicState topic2 = new StopReplicaTopicState()
|
||||
.setTopicName("topic2")
|
||||
.setPartitionStates(singletonList(new StopReplicaPartitionState()
|
||||
.setPartitionIndex(1)
|
||||
.setLeaderEpoch(2)
|
||||
.setDeletePartition(deletePartitions)));
|
||||
topicStates.add(topic2);
|
||||
|
||||
return new StopReplicaRequest.Builder(version, 0, 1, 0,
|
||||
deletePartitions, topicStates).build(version);
|
||||
}
|
||||
|
||||
private StopReplicaResponse createStopReplicaResponse() {
|
||||
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(short version) {
|
||||
ControlledShutdownRequestData data = new ControlledShutdownRequestData()
|
||||
.setBrokerId(10)
|
||||
.setBrokerEpoch(0L);
|
||||
return new ControlledShutdownRequest.Builder(
|
||||
data,
|
||||
CONTROLLED_SHUTDOWN.latestVersion()).build(version);
|
||||
}
|
||||
|
||||
private ControlledShutdownResponse createControlledShutdownResponse() {
|
||||
RemainingPartition p1 = new RemainingPartition()
|
||||
.setTopicName("test2")
|
||||
.setPartitionIndex(5);
|
||||
RemainingPartition p2 = new RemainingPartition()
|
||||
.setTopicName("test1")
|
||||
.setPartitionIndex(10);
|
||||
RemainingPartitionCollection pSet = new RemainingPartitionCollection();
|
||||
pSet.add(p1);
|
||||
pSet.add(p2);
|
||||
ControlledShutdownResponseData data = new ControlledShutdownResponseData()
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
.setRemainingPartitions(pSet);
|
||||
return new ControlledShutdownResponse(data);
|
||||
}
|
||||
|
||||
private LeaderAndIsrRequest createLeaderAndIsrRequest(short version) {
|
||||
List<LeaderAndIsrPartitionState> partitionStates = new ArrayList<>();
|
||||
List<Integer> isr = asList(1, 2);
|
||||
List<Integer> replicas = asList(1, 2, 3, 4);
|
||||
partitionStates.add(new LeaderAndIsrPartitionState()
|
||||
.setTopicName("topic5")
|
||||
.setPartitionIndex(105)
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(2)
|
||||
.setLeaderEpoch(1)
|
||||
.setIsr(isr)
|
||||
.setPartitionEpoch(2)
|
||||
.setReplicas(replicas)
|
||||
.setIsNew(false));
|
||||
partitionStates.add(new LeaderAndIsrPartitionState()
|
||||
.setTopicName("topic5")
|
||||
.setPartitionIndex(1)
|
||||
.setControllerEpoch(1)
|
||||
.setLeader(1)
|
||||
.setLeaderEpoch(1)
|
||||
.setIsr(isr)
|
||||
.setPartitionEpoch(2)
|
||||
.setReplicas(replicas)
|
||||
.setIsNew(false));
|
||||
partitionStates.add(new LeaderAndIsrPartitionState()
|
||||
.setTopicName("topic20")
|
||||
.setPartitionIndex(1)
|
||||
.setControllerEpoch(1)
|
||||
.setLeader(0)
|
||||
.setLeaderEpoch(1)
|
||||
.setIsr(isr)
|
||||
.setPartitionEpoch(2)
|
||||
.setReplicas(replicas)
|
||||
.setIsNew(false));
|
||||
|
||||
Set<Node> leaders = Set.of(
|
||||
new Node(0, "test0", 1223),
|
||||
new Node(1, "test1", 1223)
|
||||
);
|
||||
|
||||
Map<String, Uuid> topicIds = new HashMap<>();
|
||||
topicIds.put("topic5", Uuid.randomUuid());
|
||||
topicIds.put("topic20", Uuid.randomUuid());
|
||||
|
||||
return new LeaderAndIsrRequest.Builder(version, 1, 10, 0,
|
||||
partitionStates, topicIds, leaders).build();
|
||||
}
|
||||
|
||||
private LeaderAndIsrResponse createLeaderAndIsrResponse(short version) {
|
||||
if (version < 5) {
|
||||
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), version);
|
||||
} else {
|
||||
List<LeaderAndIsrResponseData.LeaderAndIsrPartitionError> partition = singletonList(
|
||||
new LeaderAndIsrResponseData.LeaderAndIsrPartitionError()
|
||||
.setPartitionIndex(0)
|
||||
.setErrorCode(Errors.NONE.code()));
|
||||
LeaderAndIsrTopicErrorCollection topics = new LeaderAndIsrTopicErrorCollection();
|
||||
topics.add(new LeaderAndIsrResponseData.LeaderAndIsrTopicError()
|
||||
.setTopicId(Uuid.randomUuid())
|
||||
.setPartitionErrors(partition));
|
||||
return new LeaderAndIsrResponse(new LeaderAndIsrResponseData()
|
||||
.setTopics(topics), version);
|
||||
}
|
||||
}
|
||||
|
||||
private UpdateMetadataRequest createUpdateMetadataRequest(short version, String rack) {
|
||||
List<UpdateMetadataPartitionState> partitionStates = new ArrayList<>();
|
||||
List<Integer> isr = asList(1, 2);
|
||||
List<Integer> replicas = asList(1, 2, 3, 4);
|
||||
List<Integer> offlineReplicas = emptyList();
|
||||
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));
|
||||
|
||||
Map<String, Uuid> topicIds = new HashMap<>();
|
||||
if (version > 6) {
|
||||
topicIds.put("topic5", Uuid.randomUuid());
|
||||
topicIds.put("topic20", Uuid.randomUuid());
|
||||
}
|
||||
|
||||
SecurityProtocol plaintext = SecurityProtocol.PLAINTEXT;
|
||||
List<UpdateMetadataEndpoint> endpoints1 = new ArrayList<>();
|
||||
endpoints1.add(new UpdateMetadataEndpoint()
|
||||
.setHost("host1")
|
||||
.setPort(1223)
|
||||
.setSecurityProtocol(plaintext.id)
|
||||
.setListener(ListenerName.forSecurityProtocol(plaintext).value()));
|
||||
|
||||
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 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");
|
||||
}
|
||||
|
||||
List<UpdateMetadataBroker> liveBrokers = asList(
|
||||
new UpdateMetadataBroker()
|
||||
.setId(0)
|
||||
.setEndpoints(endpoints1)
|
||||
.setRack(rack),
|
||||
new UpdateMetadataBroker()
|
||||
.setId(1)
|
||||
.setEndpoints(endpoints2)
|
||||
.setRack(rack)
|
||||
);
|
||||
return new UpdateMetadataRequest.Builder(version, 1, 10, 0, partitionStates,
|
||||
liveBrokers, topicIds).build();
|
||||
}
|
||||
|
||||
private UpdateMetadataResponse createUpdateMetadataResponse() {
|
||||
return new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.NONE.code()));
|
||||
}
|
||||
|
||||
private SaslHandshakeRequest createSaslHandshakeRequest(short version) {
|
||||
return new SaslHandshakeRequest.Builder(
|
||||
new SaslHandshakeRequestData().setMechanism("PLAIN")).build(version);
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicIdPartition, T
|
|||
import org.apache.kafka.common.errors._
|
||||
import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState
|
||||
import org.apache.kafka.common.message.{DescribeProducersResponseData, FetchResponseData}
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
|
||||
|
@ -327,7 +326,7 @@ class Partition(val topicPartition: TopicPartition,
|
|||
|
||||
// lock to prevent the follower replica log update while checking if the log dir could be replaced with future log.
|
||||
private val futureLogLock = new Object()
|
||||
// The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers.
|
||||
// The current epoch for the partition for KRaft controllers.
|
||||
@volatile private var partitionEpoch: Int = LeaderAndIsr.INITIAL_PARTITION_EPOCH
|
||||
@volatile private var leaderEpoch: Int = LeaderAndIsr.INITIAL_LEADER_EPOCH - 1
|
||||
// start offset for 'leaderEpoch' above (leader epoch of the current leader for this partition),
|
||||
|
@ -726,7 +725,7 @@ class Partition(val topicPartition: TopicPartition,
|
|||
* from the time when this broker was the leader last time) and setting the new leader and ISR.
|
||||
* If the leader replica id does not change, return false to indicate the replica manager.
|
||||
*/
|
||||
def makeLeader(partitionState: LeaderAndIsrPartitionState,
|
||||
def makeLeader(partitionState: LeaderAndIsrRequest.PartitionState,
|
||||
highWatermarkCheckpoints: OffsetCheckpoints,
|
||||
topicId: Option[Uuid],
|
||||
targetDirectoryId: Option[Uuid] = None): Boolean = {
|
||||
|
@ -832,7 +831,7 @@ class Partition(val topicPartition: TopicPartition,
|
|||
* replica manager that state is already correct and the become-follower steps can
|
||||
* be skipped.
|
||||
*/
|
||||
def makeFollower(partitionState: LeaderAndIsrPartitionState,
|
||||
def makeFollower(partitionState: LeaderAndIsrRequest.PartitionState,
|
||||
highWatermarkCheckpoints: OffsetCheckpoints,
|
||||
topicId: Option[Uuid],
|
||||
targetLogDirectoryId: Option[Uuid] = None): Boolean = {
|
||||
|
@ -880,7 +879,7 @@ class Partition(val topicPartition: TopicPartition,
|
|||
}
|
||||
}
|
||||
|
||||
private def createLogInAssignedDirectoryId(partitionState: LeaderAndIsrPartitionState, highWatermarkCheckpoints: OffsetCheckpoints, topicId: Option[Uuid], targetLogDirectoryId: Option[Uuid]): Unit = {
|
||||
private def createLogInAssignedDirectoryId(partitionState: LeaderAndIsrRequest.PartitionState, highWatermarkCheckpoints: OffsetCheckpoints, topicId: Option[Uuid], targetLogDirectoryId: Option[Uuid]): Unit = {
|
||||
targetLogDirectoryId match {
|
||||
case Some(directoryId) =>
|
||||
if (logManager.onlineLogDirId(directoryId) || !logManager.hasOfflineLogDirs() || directoryId == DirectoryId.UNASSIGNED) {
|
||||
|
|
|
@ -35,7 +35,6 @@ import scala.jdk.CollectionConverters._
|
|||
import scala.collection._
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.util.{Failure, Success, Try}
|
||||
import org.apache.kafka.common.requests.{AbstractControlRequest, LeaderAndIsrRequest}
|
||||
import org.apache.kafka.image.TopicsImage
|
||||
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils}
|
||||
|
||||
|
@ -1614,48 +1613,4 @@ object LogManager {
|
|||
true
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Find logs which should not be on the current broker, according to the full LeaderAndIsrRequest.
|
||||
*
|
||||
* @param brokerId The ID of the current broker.
|
||||
* @param request The full LeaderAndIsrRequest, containing all partitions owned by the broker.
|
||||
* @param logs A collection of Log objects.
|
||||
*
|
||||
* @return The topic partitions which are no longer needed on this broker.
|
||||
*/
|
||||
def findStrayReplicas(
|
||||
brokerId: Int,
|
||||
request: LeaderAndIsrRequest,
|
||||
logs: Iterable[UnifiedLog]
|
||||
): Iterable[TopicPartition] = {
|
||||
if (request.requestType() != AbstractControlRequest.Type.FULL) {
|
||||
throw new RuntimeException("Cannot use incremental LeaderAndIsrRequest to find strays.")
|
||||
}
|
||||
val partitions = new util.HashMap[TopicPartition, Uuid]()
|
||||
request.data().topicStates().forEach(topicState => {
|
||||
topicState.partitionStates().forEach(partition => {
|
||||
partitions.put(new TopicPartition(topicState.topicName(), partition.partitionIndex()),
|
||||
topicState.topicId())
|
||||
})
|
||||
})
|
||||
logs.flatMap { log =>
|
||||
val topicId = log.topicId.getOrElse {
|
||||
throw new RuntimeException(s"The log dir $log does not have a topic ID, " +
|
||||
"which is not allowed when running in KRaft mode.")
|
||||
}
|
||||
Option(partitions.get(log.topicPartition)) match {
|
||||
case Some(id) =>
|
||||
if (id.equals(topicId)) {
|
||||
None
|
||||
} else {
|
||||
info(s"Found stray log dir $log: this partition now exists with topic ID $id not $topicId.")
|
||||
Some(log.topicPartition)
|
||||
}
|
||||
case None =>
|
||||
info(s"Found stray log dir $log: this partition does not exist in the new full LeaderAndIsrRequest.")
|
||||
Some(log.topicPartition)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -803,9 +803,9 @@ private[kafka] object Processor {
|
|||
if (apiVersionManager.isApiEnabled(header.apiKey, header.apiVersion)) {
|
||||
header
|
||||
} else if (header.isApiVersionSupported()) {
|
||||
throw new InvalidRequestException(s"Received request api key ${header.apiKey} with version ${header.apiVersion} which is not enabled")
|
||||
throw new InvalidRequestException(s"Received request for disabled api with key ${header.apiKey.id} (${header.apiKey().name}) and version ${header.apiVersion}")
|
||||
} else {
|
||||
throw new UnsupportedVersionException(s"Received request api key ${header.apiKey} with version ${header.apiVersion} which is not supported")
|
||||
throw new UnsupportedVersionException(s"Received request for api with key ${header.apiKey.id} (${header.apiKey().name}) and unsupported version ${header.apiVersion}")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -182,7 +182,7 @@ class AddPartitionsToTxnManager(
|
|||
}
|
||||
|
||||
private def getTransactionCoordinator(partition: Int): Option[Node] = {
|
||||
metadataCache.getPartitionInfo(Topic.TRANSACTION_STATE_TOPIC_NAME, partition)
|
||||
metadataCache.getLeaderAndIsr(Topic.TRANSACTION_STATE_TOPIC_NAME, partition)
|
||||
.filter(_.leader != MetadataResponse.NO_LEADER_ID)
|
||||
.flatMap(metadata => metadataCache.getAliveBrokerNode(metadata.leader, interBrokerListenerName))
|
||||
}
|
||||
|
|
|
@ -74,7 +74,7 @@ class DelayedElectLeader(
|
|||
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
|
||||
case (tp, leader) if metadataCache.getLeaderAndIsr(tp.topic, tp.partition).exists(_.leader == leader) => tp
|
||||
}
|
||||
completedPartitions.foreach { tp =>
|
||||
waitingPartitions -= tp
|
||||
|
|
|
@ -298,7 +298,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val topicWithValidPartitions = new OffsetCommitRequestData.OffsetCommitRequestTopic().setName(topic.name)
|
||||
|
||||
topic.partitions.forEach { partition =>
|
||||
if (metadataCache.getPartitionInfo(topic.name, partition.partitionIndex).nonEmpty) {
|
||||
if (metadataCache.getLeaderAndIsr(topic.name, partition.partitionIndex).nonEmpty) {
|
||||
topicWithValidPartitions.partitions.add(partition)
|
||||
} else {
|
||||
responseBuilder.addPartition(topic.name, partition.partitionIndex, Errors.UNKNOWN_TOPIC_OR_PARTITION)
|
||||
|
@ -364,7 +364,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
(x.leaderReplicaIdOpt.getOrElse(-1), x.getLeaderEpoch)
|
||||
case Left(x) =>
|
||||
debug(s"Unable to retrieve local leaderId and Epoch with error $x, falling back to metadata cache")
|
||||
metadataCache.getPartitionInfo(tp.topic, tp.partition) match {
|
||||
metadataCache.getLeaderAndIsr(tp.topic, tp.partition) match {
|
||||
case Some(pinfo) => (pinfo.leader(), pinfo.leaderEpoch())
|
||||
case None => (-1, -1)
|
||||
}
|
||||
|
@ -2004,7 +2004,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val topicWithValidPartitions = new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic().setName(topic.name)
|
||||
|
||||
topic.partitions.forEach { partition =>
|
||||
if (metadataCache.getPartitionInfo(topic.name, partition.partitionIndex).nonEmpty) {
|
||||
if (metadataCache.getLeaderAndIsr(topic.name, partition.partitionIndex).nonEmpty) {
|
||||
topicWithValidPartitions.partitions.add(partition)
|
||||
} else {
|
||||
responseBuilder.addPartition(topic.name, partition.partitionIndex, Errors.UNKNOWN_TOPIC_OR_PARTITION)
|
||||
|
@ -2325,7 +2325,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val topicWithValidPartitions = new OffsetDeleteRequestData.OffsetDeleteRequestTopic().setName(topic.name)
|
||||
|
||||
topic.partitions.forEach { partition =>
|
||||
if (metadataCache.getPartitionInfo(topic.name, partition.partitionIndex).nonEmpty) {
|
||||
if (metadataCache.getLeaderAndIsr(topic.name, partition.partitionIndex).nonEmpty) {
|
||||
topicWithValidPartitions.partitions.add(partition)
|
||||
} else {
|
||||
responseBuilder.addPartition(topic.name, partition.partitionIndex, Errors.UNKNOWN_TOPIC_OR_PARTITION)
|
||||
|
|
|
@ -19,9 +19,10 @@ package kafka.server
|
|||
|
||||
import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache}
|
||||
import org.apache.kafka.admin.BrokerMetadata
|
||||
import org.apache.kafka.common.message.{DescribeClientQuotasRequestData, DescribeClientQuotasResponseData, DescribeUserScramCredentialsRequestData, DescribeUserScramCredentialsResponseData, MetadataResponseData, UpdateMetadataRequestData}
|
||||
import org.apache.kafka.common.message.{DescribeClientQuotasRequestData, DescribeClientQuotasResponseData, DescribeUserScramCredentialsRequestData, DescribeUserScramCredentialsResponseData, MetadataResponseData}
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.{Cluster, Node, TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderAndIsr
|
||||
import org.apache.kafka.server.common.{FinalizedFeatures, KRaftVersion, MetadataVersion}
|
||||
|
||||
import java.util
|
||||
|
@ -65,7 +66,7 @@ trait MetadataCache extends ConfigRepository {
|
|||
|
||||
def getBrokerNodes(listenerName: ListenerName): Iterable[Node]
|
||||
|
||||
def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataRequestData.UpdateMetadataPartitionState]
|
||||
def getLeaderAndIsr(topic: String, partitionId: Int): Option[LeaderAndIsr]
|
||||
|
||||
/**
|
||||
* Return the number of partitions in the given topic, or None if the given topic does not exist.
|
||||
|
|
|
@ -23,21 +23,18 @@ import kafka.log.remote.RemoteLogManager
|
|||
import kafka.log.{LogManager, UnifiedLog}
|
||||
import kafka.server.HostedPartition.Online
|
||||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
import kafka.server.ReplicaManager.{AtMinIsrPartitionCountMetricName, FailedIsrUpdatesPerSecMetricName, IsrExpandsPerSecMetricName, IsrShrinksPerSecMetricName, LeaderCountMetricName, OfflineReplicaCountMetricName, PartitionCountMetricName, PartitionsWithLateTransactionsCountMetricName, ProducerIdCountMetricName, ReassigningPartitionsMetricName, UnderMinIsrPartitionCountMetricName, UnderReplicatedPartitionsMetricName, createLogReadResult, isListOffsetsTimestampUnsupported}
|
||||
import kafka.server.ReplicaManager.{AtMinIsrPartitionCountMetricName, FailedIsrUpdatesPerSecMetricName, IsrExpandsPerSecMetricName, IsrShrinksPerSecMetricName, LeaderCountMetricName, OfflineReplicaCountMetricName, PartitionCountMetricName, PartitionsWithLateTransactionsCountMetricName, ProducerIdCountMetricName, ReassigningPartitionsMetricName, StopReplicaPartitionState, UnderMinIsrPartitionCountMetricName, UnderReplicatedPartitionsMetricName, createLogReadResult, isListOffsetsTimestampUnsupported}
|
||||
import kafka.server.share.DelayedShareFetch
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.errors._
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult
|
||||
import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseData.{LeaderAndIsrPartitionError, LeaderAndIsrTopicError}
|
||||
import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
|
||||
import org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse, ListOffsetsTopicResponse}
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult}
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState
|
||||
import org.apache.kafka.common.message.{DescribeLogDirsResponseData, DescribeProducersResponseData, FetchResponseData, LeaderAndIsrResponseData}
|
||||
import org.apache.kafka.common.message.{DescribeLogDirsResponseData, DescribeProducersResponseData, FetchResponseData}
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
|
@ -261,6 +258,8 @@ object ReplicaManager {
|
|||
timestamp < 0 &&
|
||||
(!timestampMinSupportedVersion.contains(timestamp) || version < timestampMinSupportedVersion(timestamp))
|
||||
}
|
||||
|
||||
case class StopReplicaPartitionState(leaderEpoch: Int, deletePartition: Boolean)
|
||||
}
|
||||
|
||||
class ReplicaManager(val config: KafkaConfig,
|
||||
|
@ -419,42 +418,6 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
brokerTopicStats.removeMetrics(topic)
|
||||
}
|
||||
|
||||
private[server] def updateStrayLogs(strayPartitions: Iterable[TopicPartition]): Unit = {
|
||||
if (strayPartitions.isEmpty) {
|
||||
return
|
||||
}
|
||||
warn(s"Found stray partitions ${strayPartitions.mkString(",")}")
|
||||
|
||||
// First, stop the partitions. This will shutdown the fetchers and other managers
|
||||
val partitionsToStop = strayPartitions.map(tp => new StopPartition(tp, false, false, false)).toSet
|
||||
stopPartitions(partitionsToStop).foreachEntry { (topicPartition, exception) =>
|
||||
error(s"Unable to stop stray partition $topicPartition", exception)
|
||||
}
|
||||
|
||||
// Next, delete the in-memory partition state. Normally, stopPartitions would do this, but since we're not
|
||||
// actually deleting the log, so we can't rely on the "deleteLocalLog" behavior in stopPartitions.
|
||||
strayPartitions.foreach { topicPartition =>
|
||||
getPartition(topicPartition) match {
|
||||
case hostedPartition: HostedPartition.Online =>
|
||||
if (allPartitions.remove(topicPartition, hostedPartition)) {
|
||||
maybeRemoveTopicMetrics(topicPartition.topic)
|
||||
hostedPartition.partition.delete()
|
||||
}
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
|
||||
// Mark the log as stray in-memory and rename the directory
|
||||
strayPartitions.foreach { tp =>
|
||||
logManager.getLog(tp).foreach(logManager.addStrayLog(tp, _))
|
||||
logManager.getLog(tp, isFuture = true).foreach(logManager.addStrayLog(tp, _))
|
||||
}
|
||||
logManager.asyncDelete(strayPartitions, isStray = true, (topicPartition, e) => {
|
||||
error(s"Failed to delete stray partition $topicPartition due to " +
|
||||
s"${e.getClass.getName} exception: ${e.getMessage}")
|
||||
})
|
||||
}
|
||||
|
||||
private def completeDelayedOperationsWhenNotPartitionLeader(topicPartition: TopicPartition, topicId: Option[Uuid]): Unit = {
|
||||
val topicPartitionOperationKey = new TopicPartitionOperationKey(topicPartition)
|
||||
delayedProducePurgatory.checkAndComplete(topicPartitionOperationKey)
|
||||
|
@ -545,7 +508,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
|
||||
val stoppedPartitions = mutable.Buffer.empty[StopPartition]
|
||||
partitionStates.foreachEntry { (topicPartition, partitionState) =>
|
||||
val deletePartition = partitionState.deletePartition()
|
||||
val deletePartition = partitionState.deletePartition
|
||||
|
||||
getPartition(topicPartition) match {
|
||||
case HostedPartition.Offline(_) =>
|
||||
|
@ -2064,14 +2027,14 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from controller $controllerId with " +
|
||||
s"correlation id $correlationId since its controller epoch ${leaderAndIsrRequest.controllerEpoch} is old. " +
|
||||
s"Latest known controller epoch is $controllerEpoch")
|
||||
leaderAndIsrRequest.getErrorResponse(0, Errors.STALE_CONTROLLER_EPOCH.exception)
|
||||
leaderAndIsrRequest.getErrorResponse(Errors.STALE_CONTROLLER_EPOCH.exception)
|
||||
} else {
|
||||
val responseMap = new mutable.HashMap[TopicPartition, Errors]
|
||||
controllerEpoch = leaderAndIsrRequest.controllerEpoch
|
||||
|
||||
val partitions = new mutable.HashSet[Partition]()
|
||||
val partitionsToBeLeader = new mutable.HashMap[Partition, LeaderAndIsrPartitionState]()
|
||||
val partitionsToBeFollower = new mutable.HashMap[Partition, LeaderAndIsrPartitionState]()
|
||||
val partitionsToBeLeader = new mutable.HashMap[Partition, LeaderAndIsrRequest.PartitionState]()
|
||||
val partitionsToBeFollower = new mutable.HashMap[Partition, LeaderAndIsrRequest.PartitionState]()
|
||||
val topicIdUpdateFollowerPartitions = new mutable.HashSet[Partition]()
|
||||
val allTopicPartitionsInRequest = new mutable.HashSet[TopicPartition]()
|
||||
|
||||
|
@ -2109,7 +2072,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
s" match the topic ID for partition $topicPartition received: " +
|
||||
s"${requestTopicId.get}.")
|
||||
responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID)
|
||||
} else if (requestLeaderEpoch > currentLeaderEpoch || (requestLeaderEpoch == currentLeaderEpoch && leaderAndIsrRequest.isKRaftController)) {
|
||||
} else 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 (partitionState.replicas.contains(localBrokerId)) {
|
||||
|
@ -2196,28 +2159,17 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
|
||||
onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower)
|
||||
|
||||
val data = new LeaderAndIsrResponseData().setErrorCode(Errors.NONE.code)
|
||||
if (leaderAndIsrRequest.version < 5) {
|
||||
responseMap.foreachEntry { (tp, error) =>
|
||||
data.partitionErrors.add(new LeaderAndIsrPartitionError()
|
||||
.setTopicName(tp.topic)
|
||||
.setPartitionIndex(tp.partition)
|
||||
.setErrorCode(error.code))
|
||||
}
|
||||
} else {
|
||||
responseMap.foreachEntry { (tp, error) =>
|
||||
val topicId = topicIds.get(tp.topic)
|
||||
var topic = data.topics.find(topicId)
|
||||
if (topic == null) {
|
||||
topic = new LeaderAndIsrTopicError().setTopicId(topicId)
|
||||
data.topics.add(topic)
|
||||
}
|
||||
topic.partitionErrors.add(new LeaderAndIsrPartitionError()
|
||||
.setPartitionIndex(tp.partition)
|
||||
.setErrorCode(error.code))
|
||||
val topics = new util.LinkedHashMap[Uuid, util.List[LeaderAndIsrResponse.PartitionError]]
|
||||
responseMap.foreachEntry { (tp, error) =>
|
||||
val topicId = topicIds.get(tp.topic)
|
||||
var partitionErrors = topics.get(topicId)
|
||||
if (partitionErrors == null) {
|
||||
partitionErrors = new util.ArrayList[LeaderAndIsrResponse.PartitionError]()
|
||||
topics.put(topicId, partitionErrors)
|
||||
}
|
||||
partitionErrors.add(new LeaderAndIsrResponse.PartitionError(tp.partition(), error.code))
|
||||
}
|
||||
new LeaderAndIsrResponse(data, leaderAndIsrRequest.version)
|
||||
new LeaderAndIsrResponse(Errors.NONE, topics)
|
||||
}
|
||||
}
|
||||
val endMs = time.milliseconds()
|
||||
|
@ -2307,7 +2259,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
*/
|
||||
private def makeLeaders(controllerId: Int,
|
||||
controllerEpoch: Int,
|
||||
partitionStates: Map[Partition, LeaderAndIsrPartitionState],
|
||||
partitionStates: Map[Partition, LeaderAndIsrRequest.PartitionState],
|
||||
correlationId: Int,
|
||||
responseMap: mutable.Map[TopicPartition, Errors],
|
||||
highWatermarkCheckpoints: OffsetCheckpoints,
|
||||
|
@ -2389,7 +2341,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
*/
|
||||
private def makeFollowers(controllerId: Int,
|
||||
controllerEpoch: Int,
|
||||
partitionStates: Map[Partition, LeaderAndIsrPartitionState],
|
||||
partitionStates: Map[Partition, LeaderAndIsrRequest.PartitionState],
|
||||
correlationId: Int,
|
||||
responseMap: mutable.Map[TopicPartition, Errors],
|
||||
highWatermarkCheckpoints: OffsetCheckpoints,
|
||||
|
|
|
@ -26,13 +26,12 @@ import org.apache.kafka.common.errors.InvalidTopicException
|
|||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData.{Cursor, DescribeTopicPartitionsResponsePartition, DescribeTopicPartitionsResponseTopic}
|
||||
import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic}
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState
|
||||
import org.apache.kafka.common.message._
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.MetadataResponse
|
||||
import org.apache.kafka.image.MetadataImage
|
||||
import org.apache.kafka.metadata.{BrokerRegistration, PartitionRegistration, Replicas}
|
||||
import org.apache.kafka.metadata.{BrokerRegistration, LeaderAndIsr, PartitionRegistration, Replicas}
|
||||
import org.apache.kafka.server.common.{FinalizedFeatures, KRaftVersion, MetadataVersion}
|
||||
|
||||
import java.util
|
||||
|
@ -383,19 +382,11 @@ class KRaftMetadataCache(
|
|||
_currentImage.cluster().brokers().values().asScala.flatMap(_.node(listenerName.value()).toScala).toSeq
|
||||
}
|
||||
|
||||
// Does NOT include offline replica metadata
|
||||
override def getPartitionInfo(topicName: String, partitionId: Int): Option[UpdateMetadataPartitionState] = {
|
||||
override def getLeaderAndIsr(topicName: String, partitionId: Int): Option[LeaderAndIsr] = {
|
||||
Option(_currentImage.topics().getTopic(topicName)).
|
||||
flatMap(topic => Option(topic.partitions().get(partitionId))).
|
||||
flatMap(partition => Some(new UpdateMetadataPartitionState().
|
||||
setTopicName(topicName).
|
||||
setPartitionIndex(partitionId).
|
||||
setControllerEpoch(-1). // Controller epoch is not stored in the cache.
|
||||
setLeader(partition.leader).
|
||||
setLeaderEpoch(partition.leaderEpoch).
|
||||
setIsr(Replicas.toList(partition.isr)).
|
||||
setZkVersion(partition.partitionEpoch).
|
||||
setReplicas(Replicas.toList(partition.replicas))))
|
||||
flatMap(partition => Some(new LeaderAndIsr(partition.leader, partition.leaderEpoch,
|
||||
util.Arrays.asList(partition.isr.map(i => i: java.lang.Integer): _*), partition.leaderRecoveryState, partition.partitionEpoch)))
|
||||
}
|
||||
|
||||
override def numPartitions(topicName: String): Option[Int] = {
|
||||
|
|
|
@ -31,8 +31,6 @@ import org.apache.kafka.common.memory.MemoryPool;
|
|||
import org.apache.kafka.common.message.DescribeTopicPartitionsRequestData;
|
||||
import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData;
|
||||
import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData.DescribeTopicPartitionsResponseTopic;
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData;
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker;
|
||||
import org.apache.kafka.common.metadata.PartitionRecord;
|
||||
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
|
||||
import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint;
|
||||
|
@ -106,16 +104,13 @@ class DescribeTopicPartitionsRequestHandlerTest {
|
|||
};
|
||||
|
||||
ListenerName plaintextListener = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT);
|
||||
UpdateMetadataBroker broker = new UpdateMetadataBroker()
|
||||
.setId(0)
|
||||
.setRack("rack")
|
||||
.setEndpoints(Arrays.asList(
|
||||
new UpdateMetadataRequestData.UpdateMetadataEndpoint()
|
||||
.setHost("broker0")
|
||||
.setPort(9092)
|
||||
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
|
||||
.setListener(plaintextListener.value())
|
||||
));
|
||||
String rack = "rack";
|
||||
int brokerId = 0;
|
||||
BrokerEndpoint brokerEndpoint = new BrokerEndpoint()
|
||||
.setName(plaintextListener.value())
|
||||
.setHost("broker0")
|
||||
.setPort(9092)
|
||||
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id);
|
||||
|
||||
@Test
|
||||
void testDescribeTopicPartitionsRequest() {
|
||||
|
@ -150,19 +145,14 @@ class DescribeTopicPartitionsRequestHandlerTest {
|
|||
topicIds.put(unauthorizedTopic, unauthorizedTopicId);
|
||||
|
||||
BrokerEndpointCollection collection = new BrokerEndpointCollection();
|
||||
collection.add(new BrokerEndpoint()
|
||||
.setName(broker.endpoints().get(0).listener())
|
||||
.setHost(broker.endpoints().get(0).host())
|
||||
.setPort(broker.endpoints().get(0).port())
|
||||
.setSecurityProtocol(broker.endpoints().get(0).securityProtocol())
|
||||
);
|
||||
collection.add(brokerEndpoint);
|
||||
List<ApiMessage> records = Arrays.asList(
|
||||
new RegisterBrokerRecord()
|
||||
.setBrokerId(broker.id())
|
||||
.setBrokerId(brokerId)
|
||||
.setBrokerEpoch(0)
|
||||
.setIncarnationId(Uuid.randomUuid())
|
||||
.setEndPoints(collection)
|
||||
.setRack(broker.rack())
|
||||
.setRack(rack)
|
||||
.setFenced(false),
|
||||
new TopicRecord().setName(authorizedTopic).setTopicId(topicIds.get(authorizedTopic)),
|
||||
new TopicRecord().setName(unauthorizedTopic).setTopicId(topicIds.get(unauthorizedTopic)),
|
||||
|
@ -352,19 +342,14 @@ class DescribeTopicPartitionsRequestHandlerTest {
|
|||
topicIds.put(authorizedTopic2, authorizedTopicId2);
|
||||
|
||||
BrokerEndpointCollection collection = new BrokerEndpointCollection();
|
||||
collection.add(new BrokerEndpoint()
|
||||
.setName(broker.endpoints().get(0).listener())
|
||||
.setHost(broker.endpoints().get(0).host())
|
||||
.setPort(broker.endpoints().get(0).port())
|
||||
.setSecurityProtocol(broker.endpoints().get(0).securityProtocol())
|
||||
);
|
||||
collection.add(brokerEndpoint);
|
||||
List<ApiMessage> records = Arrays.asList(
|
||||
new RegisterBrokerRecord()
|
||||
.setBrokerId(broker.id())
|
||||
.setBrokerId(brokerId)
|
||||
.setBrokerEpoch(0)
|
||||
.setIncarnationId(Uuid.randomUuid())
|
||||
.setEndPoints(collection)
|
||||
.setRack(broker.rack())
|
||||
.setRack(rack)
|
||||
.setFenced(false),
|
||||
new TopicRecord().setName(authorizedTopic).setTopicId(topicIds.get(authorizedTopic)),
|
||||
new TopicRecord().setName(authorizedTopic2).setTopicId(topicIds.get(authorizedTopic2)),
|
||||
|
@ -563,4 +548,4 @@ class DescribeTopicPartitionsRequestHandlerTest {
|
|||
TestUtils.setIbpVersion(properties, MetadataVersion.latestProduction());
|
||||
return new KafkaConfig(properties);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -116,7 +116,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => {
|
||||
Errors.forCode(resp.data.coordinators.asScala.find(g => group == g.key).head.errorCode)
|
||||
}),
|
||||
ApiKeys.UPDATE_METADATA -> ((resp: requests.UpdateMetadataResponse) => resp.error),
|
||||
ApiKeys.JOIN_GROUP -> ((resp: JoinGroupResponse) => resp.error),
|
||||
ApiKeys.SYNC_GROUP -> ((resp: SyncGroupResponse) => Errors.forCode(resp.data.errorCode)),
|
||||
ApiKeys.DESCRIBE_GROUPS -> ((resp: DescribeGroupsResponse) => {
|
||||
|
@ -125,12 +124,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
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) => Errors.forCode(
|
||||
resp.topics.asScala.find(t => topicNames(t.topicId) == tp.topic).get.partitionErrors.asScala.find(
|
||||
p => 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(topic).errorCode)),
|
||||
ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => Errors.forCode(resp.data.responses.find(topic).errorCode)),
|
||||
ApiKeys.DELETE_RECORDS -> ((resp: requests.DeleteRecordsResponse) => Errors.forCode(
|
||||
|
@ -1241,7 +1234,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
waitUntilTrue(() => {
|
||||
consumer.poll(Duration.ofMillis(50L))
|
||||
brokers.forall { broker =>
|
||||
broker.metadataCache.getPartitionInfo(newTopic, 0) match {
|
||||
broker.metadataCache.getLeaderAndIsr(newTopic, 0) match {
|
||||
case Some(partitionState) => FetchRequest.isValidBrokerId(partitionState.leader)
|
||||
case _ => false
|
||||
}
|
||||
|
@ -1254,7 +1247,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
def testCreatePermissionMetadataRequestAutoCreate(quorum: String): Unit = {
|
||||
val readAcls = topicReadAcl(topicResource)
|
||||
addAndVerifyAcls(readAcls, topicResource)
|
||||
brokers.foreach(b => assertEquals(None, b.metadataCache.getPartitionInfo(topic, 0)))
|
||||
brokers.foreach(b => assertEquals(None, b.metadataCache.getLeaderAndIsr(topic, 0)))
|
||||
|
||||
val metadataRequest = new MetadataRequest.Builder(List(topic).asJava, true).build()
|
||||
val metadataResponse = connectAndReceive[MetadataResponse](metadataRequest)
|
||||
|
|
|
@ -1352,7 +1352,7 @@ class KRaftClusterTest {
|
|||
// Shut down broker0 and wait until the ISR of foo-0 is set to [1, 2]
|
||||
broker0.shutdown()
|
||||
TestUtils.retry(60000) {
|
||||
val info = broker1.metadataCache.getPartitionInfo("foo", 0)
|
||||
val info = broker1.metadataCache.getLeaderAndIsr("foo", 0)
|
||||
assertTrue(info.isDefined)
|
||||
assertEquals(Set(1, 2), info.get.isr().asScala.toSet)
|
||||
}
|
||||
|
@ -1366,7 +1366,7 @@ class KRaftClusterTest {
|
|||
// Start up broker0 and wait until the ISR of foo-0 is set to [0, 1, 2]
|
||||
broker0.startup()
|
||||
TestUtils.retry(60000) {
|
||||
val info = broker1.metadataCache.getPartitionInfo("foo", 0)
|
||||
val info = broker1.metadataCache.getLeaderAndIsr("foo", 0)
|
||||
assertTrue(info.isDefined)
|
||||
assertEquals(Set(0, 1, 2), info.get.isr().asScala.toSet)
|
||||
}
|
||||
|
@ -1407,7 +1407,7 @@ class KRaftClusterTest {
|
|||
// Shut down broker0 and wait until the ISR of foo-0 is set to [1, 2]
|
||||
broker0.shutdown()
|
||||
TestUtils.retry(60000) {
|
||||
val info = broker1.metadataCache.getPartitionInfo("foo", 0)
|
||||
val info = broker1.metadataCache.getLeaderAndIsr("foo", 0)
|
||||
assertTrue(info.isDefined)
|
||||
assertEquals(Set(1, 2), info.get.isr().asScala.toSet)
|
||||
}
|
||||
|
@ -1421,7 +1421,7 @@ class KRaftClusterTest {
|
|||
// Start up broker0 and wait until the ISR of foo-0 is set to [0, 1, 2]
|
||||
broker0.startup()
|
||||
TestUtils.retry(60000) {
|
||||
val info = broker1.metadataCache.getPartitionInfo("foo", 0)
|
||||
val info = broker1.metadataCache.getLeaderAndIsr("foo", 0)
|
||||
assertTrue(info.isDefined)
|
||||
assertEquals(Set(0, 1, 2), info.get.isr().asScala.toSet)
|
||||
assertTrue(broker0.logManager.getLog(foo0, isFuture = true).isEmpty)
|
||||
|
@ -1472,7 +1472,7 @@ class KRaftClusterTest {
|
|||
// Shut down broker0 and wait until the ISR of foo-0 is set to [1, 2]
|
||||
broker0.shutdown()
|
||||
TestUtils.retry(60000) {
|
||||
val info = broker1.metadataCache.getPartitionInfo("foo", 0)
|
||||
val info = broker1.metadataCache.getLeaderAndIsr("foo", 0)
|
||||
assertTrue(info.isDefined)
|
||||
assertEquals(Set(1, 2), info.get.isr().asScala.toSet)
|
||||
}
|
||||
|
@ -1498,7 +1498,7 @@ class KRaftClusterTest {
|
|||
// Start up broker0 and wait until the ISR of foo-0 is set to [0, 1, 2]
|
||||
broker0.startup()
|
||||
TestUtils.retry(60000) {
|
||||
val info = broker1.metadataCache.getPartitionInfo("foo", 0)
|
||||
val info = broker1.metadataCache.getLeaderAndIsr("foo", 0)
|
||||
assertTrue(info.isDefined)
|
||||
assertEquals(Set(0, 1, 2), info.get.isr().asScala.toSet)
|
||||
assertTrue(broker0.logManager.getLog(foo0, isFuture = true).isEmpty)
|
||||
|
|
|
@ -21,11 +21,10 @@ import kafka.server.QuotaFactory.QuotaManagers
|
|||
import kafka.utils.{CoreUtils, Logging, TestUtils}
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.{Node, TopicPartition, Uuid}
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest
|
||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||
|
@ -223,8 +222,8 @@ class LocalLeaderEndPointTest extends Logging {
|
|||
private def buildLeaderAndIsrRequest(leaderEpoch: Int): LeaderAndIsrRequest = {
|
||||
val brokerList = Seq[Integer](sourceBroker.id).asJava
|
||||
val topicIds = Collections.singletonMap(topic, topicId)
|
||||
new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, 0,
|
||||
Seq(new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.Builder(0, 0, 0,
|
||||
Seq(new LeaderAndIsrRequest.PartitionState()
|
||||
.setTopicName(topic)
|
||||
.setPartitionIndex(topicPartition.partition())
|
||||
.setControllerEpoch(0)
|
||||
|
|
|
@ -23,7 +23,7 @@ import kafka.utils.TestUtils
|
|||
import kafka.utils.TestUtils.MockAlterPartitionManager
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid}
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.config.ReplicationConfigs
|
||||
|
@ -126,7 +126,7 @@ class AbstractPartitionTest {
|
|||
val isr = replicas
|
||||
|
||||
if (isLeader) {
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -136,7 +136,7 @@ class AbstractPartitionTest {
|
|||
.setIsNew(true), offsetCheckpoints, None), "Expected become leader transition to succeed")
|
||||
assertEquals(leaderEpoch, partition.getLeaderEpoch)
|
||||
} else {
|
||||
assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeFollower(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(remoteReplicaId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
package kafka.cluster
|
||||
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
|
||||
|
@ -89,7 +89,7 @@ class AssignmentStateTest extends AbstractPartitionTest {
|
|||
original: Seq[Int], isUnderReplicated: Boolean): Unit = {
|
||||
val controllerEpoch = 3
|
||||
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(6)
|
||||
|
|
|
@ -27,10 +27,9 @@ import kafka.server.metadata.MockConfigRepository
|
|||
import kafka.utils._
|
||||
import org.apache.kafka.common.TopicIdPartition
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
|
||||
import org.apache.kafka.common.requests.FetchRequest
|
||||
import org.apache.kafka.common.requests.{FetchRequest, LeaderAndIsrRequest}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid}
|
||||
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
|
||||
|
@ -144,7 +143,7 @@ class PartitionLockTest extends Logging {
|
|||
val replicaToCheck = 3
|
||||
val firstReplicaSet = Seq[Integer](3, 4, 5).asJava
|
||||
val secondReplicaSet = Seq[Integer](1, 2, 3).asJava
|
||||
def partitionState(replicas: java.util.List[Integer]) = new LeaderAndIsrPartitionState()
|
||||
def partitionState(replicas: java.util.List[Integer]) = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(1)
|
||||
.setLeader(replicas.get(0))
|
||||
.setLeaderEpoch(1)
|
||||
|
@ -350,7 +349,7 @@ class PartitionLockTest extends Logging {
|
|||
val replicas = (0 to numReplicaFetchers).map(i => Integer.valueOf(brokerId + i)).toList.asJava
|
||||
val isr = replicas
|
||||
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
|
|
@ -23,11 +23,10 @@ import kafka.server._
|
|||
import kafka.utils._
|
||||
import org.apache.kafka.common.errors.{ApiException, FencedLeaderEpochException, InconsistentTopicIdException, InvalidTxnStateException, NotLeaderOrFollowerException, OffsetNotAvailableException, OffsetOutOfRangeException, UnknownLeaderEpochException}
|
||||
import org.apache.kafka.common.message.{AlterPartitionResponseData, FetchResponseData}
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
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.{AlterPartitionResponse, FetchRequest, ListOffsetsRequest, RequestHeader}
|
||||
import org.apache.kafka.common.requests.{AlterPartitionResponse, FetchRequest, LeaderAndIsrRequest, ListOffsetsRequest, RequestHeader}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicPartition, Uuid}
|
||||
import org.apache.kafka.server.config.ReplicationConfigs
|
||||
|
@ -313,7 +312,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
val partitionEpoch = 1
|
||||
|
||||
assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeFollower(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -356,7 +355,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
val partitionEpoch = 1
|
||||
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -392,7 +391,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val newPartitionEpoch = partitionEpoch + 1
|
||||
val addingReplicas = List(addingReplica1, addingReplica2)
|
||||
|
||||
assertFalse(partition.makeLeader(new LeaderAndIsrPartitionState()
|
||||
assertFalse(partition.makeLeader(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -491,7 +490,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
appendThread.start()
|
||||
TestUtils.waitUntilTrue(() => appendSemaphore.hasQueuedThreads, "follower log append is not called.")
|
||||
|
||||
val partitionState = new LeaderAndIsrPartitionState()
|
||||
val partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(2)
|
||||
.setLeaderEpoch(1)
|
||||
|
@ -790,7 +789,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
new SimpleRecord(20,"k4".getBytes, "v2".getBytes),
|
||||
new SimpleRecord(21,"k5".getBytes, "v3".getBytes)))
|
||||
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -861,7 +860,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertEquals(Right(None), fetchOffsetsForTimestamp(30, Some(IsolationLevel.READ_UNCOMMITTED)))
|
||||
|
||||
// Make into a follower
|
||||
val followerState = new LeaderAndIsrPartitionState()
|
||||
val followerState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(follower2)
|
||||
.setLeaderEpoch(leaderEpoch + 1)
|
||||
|
@ -873,7 +872,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertTrue(partition.makeFollower(followerState, offsetCheckpoints, None))
|
||||
|
||||
// Back to leader, this resets the startLogOffset for this epoch (to 2), we're now in the fault condition
|
||||
val newLeaderState = new LeaderAndIsrPartitionState()
|
||||
val newLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch + 2)
|
||||
|
@ -1006,7 +1005,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1084,7 +1083,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
@Test
|
||||
def testMakeFollowerWithNoLeaderIdChange(): Unit = {
|
||||
// Start off as follower
|
||||
var partitionState = new LeaderAndIsrPartitionState()
|
||||
var partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(1)
|
||||
.setLeaderEpoch(1)
|
||||
|
@ -1095,7 +1094,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.makeFollower(partitionState, offsetCheckpoints, None)
|
||||
|
||||
// Request with same leader and epoch increases by only 1, do become-follower steps
|
||||
partitionState = new LeaderAndIsrPartitionState()
|
||||
partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(1)
|
||||
.setLeaderEpoch(4)
|
||||
|
@ -1106,7 +1105,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertTrue(partition.makeFollower(partitionState, offsetCheckpoints, None))
|
||||
|
||||
// Request with same leader and same epoch, skip become-follower steps
|
||||
partitionState = new LeaderAndIsrPartitionState()
|
||||
partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(1)
|
||||
.setLeaderEpoch(4)
|
||||
|
@ -1133,7 +1132,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val batch3 = TestUtils.records(records = List(new SimpleRecord("k6".getBytes, "v1".getBytes),
|
||||
new SimpleRecord("k7".getBytes, "v2".getBytes)))
|
||||
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1160,7 +1159,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertEquals(lastOffsetOfFirstBatch + 1, partition.log.get.highWatermark, "Expected leader's HW")
|
||||
|
||||
// current leader becomes follower and then leader again (without any new records appended)
|
||||
val followerState = new LeaderAndIsrPartitionState()
|
||||
val followerState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(follower2)
|
||||
.setLeaderEpoch(leaderEpoch + 1)
|
||||
|
@ -1170,7 +1169,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
.setIsNew(false)
|
||||
partition.makeFollower(followerState, offsetCheckpoints, None)
|
||||
|
||||
val newLeaderState = new LeaderAndIsrPartitionState()
|
||||
val newLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch + 2)
|
||||
|
@ -1252,7 +1251,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
assertFalse(partition.isAtMinIsr)
|
||||
// Make isr set to only have leader to trigger AtMinIsr (default min isr config is 1)
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leader)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1279,7 +1278,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
alterPartitionManager)
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None)
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -1291,7 +1290,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
topicId = None)
|
||||
assertFalse(partition.isUnderMinIsr)
|
||||
|
||||
val LeaderState = new LeaderAndIsrPartitionState()
|
||||
val LeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(1)
|
||||
|
@ -1319,7 +1318,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
val initializeTimeMs = time.milliseconds()
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1383,7 +1382,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
val initializeTimeMs = time.milliseconds()
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1417,7 +1416,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1468,7 +1467,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1531,7 +1530,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1603,7 +1602,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1645,7 +1644,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
// Controller shrinks the ISR after
|
||||
assertFalse(
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1709,7 +1708,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1815,7 +1814,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1906,7 +1905,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1971,7 +1970,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -2126,7 +2125,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -2208,7 +2207,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
|
||||
assertTrue(partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -2686,7 +2685,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
val controllerEpoch = 3
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(6)
|
||||
|
@ -2704,7 +2703,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 5
|
||||
val topicId = Uuid.randomUuid()
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -2748,7 +2747,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 5
|
||||
val topicId = Uuid.randomUuid()
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val leaderState = new LeaderAndIsrPartitionState()
|
||||
val leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -2825,7 +2824,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val replicas = List[Integer](brokerId, brokerId + 1, brokerId + 2).asJava
|
||||
val isr = List[Integer](brokerId, brokerId + 1).asJava
|
||||
|
||||
var leaderState = new LeaderAndIsrPartitionState()
|
||||
var leaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(6)
|
||||
|
@ -3028,7 +3027,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
val topicId = Uuid.randomUuid()
|
||||
|
||||
val initialLeaderState = new LeaderAndIsrPartitionState()
|
||||
val initialLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3061,7 +3060,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
// makeLeader is called again with the same leader epoch but with
|
||||
// a newer partition epoch. This can happen in KRaft when a partition
|
||||
// is reassigned. The leader epoch is not bumped when we add replicas.
|
||||
val updatedLeaderState = new LeaderAndIsrPartitionState()
|
||||
val updatedLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3092,7 +3091,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
val topicId = Uuid.randomUuid()
|
||||
|
||||
val initialLeaderState = new LeaderAndIsrPartitionState()
|
||||
val initialLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3118,7 +3117,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
// makeLeader is called again with the same leader epoch but with
|
||||
// a newer partition epoch.
|
||||
val updatedLeaderState = new LeaderAndIsrPartitionState()
|
||||
val updatedLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3143,7 +3142,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
val topicId = Uuid.randomUuid()
|
||||
|
||||
val initialLeaderState = new LeaderAndIsrPartitionState()
|
||||
val initialLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3158,7 +3157,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
// makeLeader is called again with the same leader epoch but with
|
||||
// a older partition epoch.
|
||||
val updatedLeaderState = new LeaderAndIsrPartitionState()
|
||||
val updatedLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3181,7 +3180,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
val topicId = Uuid.randomUuid()
|
||||
|
||||
val initialFollowerState = new LeaderAndIsrPartitionState()
|
||||
val initialFollowerState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(followerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3196,7 +3195,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
// makeLeader is called again with the same leader epoch but with
|
||||
// a older partition epoch.
|
||||
val updatedFollowerState = new LeaderAndIsrPartitionState()
|
||||
val updatedFollowerState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(followerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3220,7 +3219,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val topicId = Uuid.randomUuid()
|
||||
|
||||
// The local replica is the leader.
|
||||
val initialLeaderState = new LeaderAndIsrPartitionState()
|
||||
val initialLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(localReplica)
|
||||
.setLeaderEpoch(1)
|
||||
|
@ -3238,7 +3237,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertEquals(replicas, partition.assignmentState.replicas)
|
||||
|
||||
// The local replica becomes a follower.
|
||||
val updatedLeaderState = new LeaderAndIsrPartitionState()
|
||||
val updatedLeaderState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(remoteReplica1)
|
||||
.setLeaderEpoch(2)
|
||||
|
@ -3261,7 +3260,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None)
|
||||
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -3331,7 +3330,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None)
|
||||
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -3352,7 +3351,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None)
|
||||
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -3394,7 +3393,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None)
|
||||
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -3418,7 +3417,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None)
|
||||
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -3444,7 +3443,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertTrue(partition.log.isDefined)
|
||||
|
||||
partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(0)
|
||||
|
@ -3514,7 +3513,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
|
||||
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
|
||||
assertTrue(partition.makeLeader(new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3579,7 +3578,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
topicId
|
||||
)
|
||||
val newLeader = partition.makeLeader(
|
||||
new LeaderAndIsrPartitionState()
|
||||
new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3761,7 +3760,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 1
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val isr = replicas
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3806,7 +3805,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 1
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val isr = replicas
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3851,7 +3850,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 1
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val isr = replicas
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3896,7 +3895,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 1
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val isr = replicas
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3941,7 +3940,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 1
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val isr = replicas
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -3987,7 +3986,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 1
|
||||
val replicas = List[Integer](brokerId, brokerId + 1).asJava
|
||||
val isr = replicas
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
|
||||
val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(controllerEpoch)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
|
|
@ -282,7 +282,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
|
|||
produceMessage(brokers, topic, "third")
|
||||
//make sure follower server joins the ISR
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
val partitionInfoOpt = followerServer.metadataCache.getPartitionInfo(topic, partitionId)
|
||||
val partitionInfoOpt = followerServer.metadataCache.getLeaderAndIsr(topic, partitionId)
|
||||
partitionInfoOpt.isDefined && partitionInfoOpt.get.isr.contains(followerId)
|
||||
}, "Inconsistent metadata after first server startup")
|
||||
|
||||
|
@ -424,9 +424,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
|
|||
}
|
||||
|
||||
private def waitForNoLeaderAndIsrHasOldLeaderId(metadataCache: MetadataCache, leaderId: Int): Unit = {
|
||||
waitUntilTrue(() => metadataCache.getPartitionInfo(topic, partitionId).isDefined &&
|
||||
metadataCache.getPartitionInfo(topic, partitionId).get.leader() == LeaderConstants.NO_LEADER &&
|
||||
java.util.Arrays.asList(leaderId).equals(metadataCache.getPartitionInfo(topic, partitionId).get.isr()),
|
||||
waitUntilTrue(() => metadataCache.getLeaderAndIsr(topic, partitionId).isDefined &&
|
||||
metadataCache.getLeaderAndIsr(topic, partitionId).get.leader() == LeaderConstants.NO_LEADER &&
|
||||
java.util.Arrays.asList(leaderId).equals(metadataCache.getLeaderAndIsr(topic, partitionId).get.isr()),
|
||||
"Timed out waiting for broker metadata cache updates the info for topic partition:" + topicPartition)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,9 +23,6 @@ import kafka.utils._
|
|||
import org.apache.directory.api.util.FileUtils
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
import org.apache.kafka.common.errors.OffsetOutOfRangeException
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrTopicState
|
||||
import org.apache.kafka.common.requests.{AbstractControlRequest, LeaderAndIsrRequest}
|
||||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
import org.apache.kafka.common.{DirectoryId, KafkaException, TopicIdPartition, TopicPartition, Uuid}
|
||||
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
|
||||
|
@ -1295,47 +1292,6 @@ class LogManagerTest {
|
|||
assertTrue(LogManager.isStrayKraftReplica(0, topicsImage(Seq()), log))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFindStrayReplicasInEmptyLAIR(): Unit = {
|
||||
val onDisk = Seq(foo0, foo1, bar0, bar1, baz0, baz1, baz2, quux0)
|
||||
val expected = onDisk.map(_.topicPartition()).toSet
|
||||
assertEquals(expected,
|
||||
LogManager.findStrayReplicas(0,
|
||||
createLeaderAndIsrRequestForStrayDetection(Seq()),
|
||||
onDisk.map(mockLog)).toSet)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFindNoStrayReplicasInFullLAIR(): Unit = {
|
||||
val onDisk = Seq(foo0, foo1, bar0, bar1, baz0, baz1, baz2, quux0)
|
||||
assertEquals(Set(),
|
||||
LogManager.findStrayReplicas(0,
|
||||
createLeaderAndIsrRequestForStrayDetection(onDisk),
|
||||
onDisk.map(mockLog)).toSet)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFindSomeStrayReplicasInFullLAIR(): Unit = {
|
||||
val onDisk = Seq(foo0, foo1, bar0, bar1, baz0, baz1, baz2, quux0)
|
||||
val present = Seq(foo0, bar0, bar1, quux0)
|
||||
val expected = Seq(foo1, baz0, baz1, baz2).map(_.topicPartition()).toSet
|
||||
assertEquals(expected,
|
||||
LogManager.findStrayReplicas(0,
|
||||
createLeaderAndIsrRequestForStrayDetection(present),
|
||||
onDisk.map(mockLog)).toSet)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testTopicRecreationInFullLAIR(): Unit = {
|
||||
val onDisk = Seq(foo0, foo1, bar0, bar1, baz0, baz1, baz2, quux0)
|
||||
val present = Seq(recreatedFoo0, recreatedFoo1, bar0, baz0, baz1, baz2, quux0)
|
||||
val expected = Seq(foo0, foo1, bar1).map(_.topicPartition()).toSet
|
||||
assertEquals(expected,
|
||||
LogManager.findStrayReplicas(0,
|
||||
createLeaderAndIsrRequestForStrayDetection(present),
|
||||
onDisk.map(mockLog)).toSet)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test LogManager takes file lock by default and the lock is released after shutdown.
|
||||
*/
|
||||
|
@ -1473,40 +1429,4 @@ object LogManagerTest {
|
|||
topics.foreach { t => retval = retval.including(t) }
|
||||
retval
|
||||
}
|
||||
|
||||
def createLeaderAndIsrRequestForStrayDetection(
|
||||
partitions: Iterable[TopicIdPartition],
|
||||
leaders: Iterable[Int] = Seq(),
|
||||
): LeaderAndIsrRequest = {
|
||||
val nextLeaderIter = leaders.iterator
|
||||
def nextLeader(): Int = {
|
||||
if (nextLeaderIter.hasNext) {
|
||||
nextLeaderIter.next()
|
||||
} else {
|
||||
3
|
||||
}
|
||||
}
|
||||
val data = new LeaderAndIsrRequestData().
|
||||
setControllerId(1000).
|
||||
setIsKRaftController(true).
|
||||
setType(AbstractControlRequest.Type.FULL.toByte)
|
||||
val topics = new java.util.LinkedHashMap[String, LeaderAndIsrTopicState]
|
||||
partitions.foreach(partition => {
|
||||
val topicState = topics.computeIfAbsent(partition.topic(),
|
||||
_ => new LeaderAndIsrTopicState().
|
||||
setTopicId(partition.topicId()).
|
||||
setTopicName(partition.topic()))
|
||||
topicState.partitionStates().add(new LeaderAndIsrRequestData.LeaderAndIsrPartitionState().
|
||||
setTopicName(partition.topic()).
|
||||
setPartitionIndex(partition.partition()).
|
||||
setControllerEpoch(123).
|
||||
setLeader(nextLeader()).
|
||||
setLeaderEpoch(456).
|
||||
setIsr(java.util.Arrays.asList(3, 4, 5)).
|
||||
setReplicas(java.util.Arrays.asList(3, 4, 5)).
|
||||
setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()))
|
||||
})
|
||||
data.topicStates().addAll(topics.values())
|
||||
new LeaderAndIsrRequest(data, 7.toShort)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,10 +20,11 @@ package kafka.network
|
|||
import kafka.server.SimpleApiVersionManager
|
||||
import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException}
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
import org.apache.kafka.common.message.RequestHeaderData
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.requests.{RequestHeader, RequestTestUtils}
|
||||
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
|
||||
import org.junit.jupiter.api.Assertions.assertThrows
|
||||
import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue}
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.function.Executable
|
||||
|
||||
|
@ -32,13 +33,33 @@ import java.util.Collections
|
|||
class ProcessorTest {
|
||||
|
||||
@Test
|
||||
def testParseRequestHeaderWithDisabledApi(): Unit = {
|
||||
def testParseRequestHeaderWithDisabledApiVersion(): Unit = {
|
||||
val requestHeader = RequestTestUtils.serializeRequestHeader(
|
||||
new RequestHeader(ApiKeys.INIT_PRODUCER_ID, 0, "clientid", 0))
|
||||
val apiVersionManager = new SimpleApiVersionManager(ListenerType.CONTROLLER, true,
|
||||
() => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true))
|
||||
assertThrows(classOf[InvalidRequestException], (() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable,
|
||||
val e = assertThrows(classOf[InvalidRequestException],
|
||||
(() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable,
|
||||
"INIT_PRODUCER_ID with listener type CONTROLLER should throw InvalidRequestException exception")
|
||||
assertTrue(e.toString.contains("disabled api"));
|
||||
}
|
||||
|
||||
@Test
|
||||
def testParseRequestHeaderWithUnsupportedApi(): Unit = {
|
||||
// We have to use `RequestHeaderData` since `ApiMessageType` doesn't support this protocol api
|
||||
val headerVersion = 0.toShort
|
||||
val requestHeaderData = new RequestHeaderData()
|
||||
.setRequestApiKey(ApiKeys.LEADER_AND_ISR.id)
|
||||
.setRequestApiVersion(headerVersion)
|
||||
.setClientId("clientid")
|
||||
.setCorrelationId(0);
|
||||
val requestHeader = RequestTestUtils.serializeRequestHeader(new RequestHeader(requestHeaderData, headerVersion))
|
||||
val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true,
|
||||
() => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true))
|
||||
val e = assertThrows(classOf[InvalidRequestException],
|
||||
(() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable,
|
||||
"LEADER_AND_ISR should throw InvalidRequestException exception")
|
||||
assertTrue(e.toString.contains("Unsupported api"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -47,8 +68,10 @@ class ProcessorTest {
|
|||
new RequestHeader(ApiKeys.PRODUCE, 0, "clientid", 0))
|
||||
val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true,
|
||||
() => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true))
|
||||
assertThrows(classOf[UnsupportedVersionException], (() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable,
|
||||
val e = assertThrows(classOf[UnsupportedVersionException],
|
||||
(() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable,
|
||||
"PRODUCE v0 should throw UnsupportedVersionException exception")
|
||||
assertTrue(e.toString.contains("unsupported version"));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -25,11 +25,11 @@ import org.apache.kafka.common.internals.Topic
|
|||
import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.{AddPartitionsToTxnTopic, AddPartitionsToTxnTopicCollection, AddPartitionsToTxnTransaction, AddPartitionsToTxnTransactionCollection}
|
||||
import org.apache.kafka.common.message.AddPartitionsToTxnResponseData
|
||||
import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnResultCollection
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState
|
||||
import org.apache.kafka.common.{Node, TopicPartition}
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.{AbstractResponse, AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, MetadataResponse}
|
||||
import org.apache.kafka.common.utils.MockTime
|
||||
import org.apache.kafka.metadata.LeaderAndIsr
|
||||
import org.apache.kafka.server.metrics.KafkaMetricsGroup
|
||||
import org.apache.kafka.server.util.RequestAndCompletionHandler
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
|
@ -41,6 +41,7 @@ import org.mockito.ArgumentMatchers.{any, anyLong, anyString}
|
|||
import org.mockito.MockedConstruction.Context
|
||||
import org.mockito.Mockito.{mock, mockConstruction, times, verify, verifyNoMoreInteractions, when}
|
||||
|
||||
import java.util
|
||||
import java.util.concurrent.TimeUnit
|
||||
import scala.collection.mutable
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -218,7 +219,7 @@ class AddPartitionsToTxnManagerTest {
|
|||
}
|
||||
|
||||
// The transaction state topic does not exist.
|
||||
when(metadataCache.getPartitionInfo(Topic.TRANSACTION_STATE_TOPIC_NAME, 0))
|
||||
when(metadataCache.getLeaderAndIsr(Topic.TRANSACTION_STATE_TOPIC_NAME, 0))
|
||||
.thenReturn(Option.empty)
|
||||
checkError()
|
||||
|
||||
|
@ -403,12 +404,8 @@ class AddPartitionsToTxnManagerTest {
|
|||
}
|
||||
|
||||
private def mockTransactionStateMetadata(partitionIndex: Int, leaderId: Int, leaderNode: Option[Node]): Unit = {
|
||||
when(metadataCache.getPartitionInfo(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionIndex))
|
||||
.thenReturn(Some(
|
||||
new UpdateMetadataPartitionState()
|
||||
.setTopicName(Topic.TRANSACTION_STATE_TOPIC_NAME)
|
||||
.setPartitionIndex(partitionIndex)
|
||||
.setLeader(leaderId)))
|
||||
when(metadataCache.getLeaderAndIsr(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionIndex))
|
||||
.thenReturn(Some(new LeaderAndIsr(leaderId, util.Arrays.asList(leaderId))))
|
||||
if (leaderId != MetadataResponse.NO_LEADER_ID) {
|
||||
when(metadataCache.getAliveBrokerNode(leaderId, config.interBrokerListenerName))
|
||||
.thenReturn(leaderNode)
|
||||
|
|
|
@ -49,8 +49,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
|
|||
|
||||
// Ensure one topic partition is offline.
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
aliveBrokers.head.metadataCache.getPartitionInfo(onlineTopic, 0).exists(_.leader() == 1) &&
|
||||
aliveBrokers.head.metadataCache.getPartitionInfo(offlineTopic, 0).exists(_.leader() ==
|
||||
aliveBrokers.head.metadataCache.getLeaderAndIsr(onlineTopic, 0).exists(_.leader() == 1) &&
|
||||
aliveBrokers.head.metadataCache.getLeaderAndIsr(offlineTopic, 0).exists(_.leader() ==
|
||||
MetadataResponse.NO_LEADER_ID)
|
||||
}, "Topic partition is not offline")
|
||||
|
||||
|
|
|
@ -2234,7 +2234,7 @@ class KafkaApisTest extends Logging {
|
|||
when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs(
|
||||
any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0)
|
||||
when(metadataCache.contains(tp)).thenAnswer(_ => true)
|
||||
when(metadataCache.getPartitionInfo(tp.topic(), tp.partition())).thenAnswer(_ => Option.empty)
|
||||
when(metadataCache.getLeaderAndIsr(tp.topic(), tp.partition())).thenAnswer(_ => Option.empty)
|
||||
when(metadataCache.getAliveBrokerNode(any(), any())).thenReturn(Option.empty)
|
||||
kafkaApis = createKafkaApis()
|
||||
kafkaApis.handleProduceRequest(request, RequestLocal.withThreadConfinedCaching)
|
||||
|
|
|
@ -784,7 +784,7 @@ class MetadataCacheTest {
|
|||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("cacheProvider"))
|
||||
def testGetPartitionInfo(cache: MetadataCache): Unit = {
|
||||
def testGetLeaderAndIsr(cache: MetadataCache): Unit = {
|
||||
val topic = "topic"
|
||||
val topicId = Uuid.randomUuid()
|
||||
val partitionIndex = 0
|
||||
|
@ -819,14 +819,12 @@ class MetadataCacheTest {
|
|||
|
||||
MetadataCacheTest.updateCache(cache, brokers ++ topicRecords ++ partitionStates)
|
||||
|
||||
val partitionState = cache.getPartitionInfo(topic, partitionIndex).get
|
||||
assertEquals(topic, partitionState.topicName())
|
||||
assertEquals(partitionIndex, partitionState.partitionIndex())
|
||||
assertEquals(-1, partitionState.controllerEpoch())
|
||||
assertEquals(leader, partitionState.leader())
|
||||
assertEquals(leaderEpoch, partitionState.leaderEpoch())
|
||||
assertEquals(isr, partitionState.isr())
|
||||
assertEquals(replicas, partitionState.replicas())
|
||||
val leaderAndIsr = cache.getLeaderAndIsr(topic, partitionIndex)
|
||||
assertEquals(Some(leader), leaderAndIsr.map(_.leader()))
|
||||
assertEquals(Some(leaderEpoch), leaderAndIsr.map(_.leaderEpoch()))
|
||||
assertEquals(Some(isr), leaderAndIsr.map(_.isr()))
|
||||
assertEquals(Some(-1), leaderAndIsr.map(_.partitionEpoch()))
|
||||
assertEquals(Some(LeaderRecoveryState.RECOVERED), leaderAndIsr.map(_.leaderRecoveryState()))
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -24,15 +24,11 @@ import org.apache.kafka.common.internals.Topic
|
|||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
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.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection}
|
||||
import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState}
|
||||
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
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.quota.ClientQuotaFilter
|
||||
import org.apache.kafka.common.record._
|
||||
|
@ -41,7 +37,6 @@ import org.apache.kafka.common.resource.{PatternType, ResourceType => AdminResou
|
|||
import org.apache.kafka.common.security.auth._
|
||||
import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils}
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.metadata.LeaderAndIsr
|
||||
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
||||
import org.apache.kafka.network.Session
|
||||
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
|
||||
|
@ -70,7 +65,6 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
private val unthrottledClientId = "unthrottled-client"
|
||||
private val smallQuotaProducerClientId = "small-quota-producer-client"
|
||||
private val smallQuotaConsumerClientId = "small-quota-consumer-client"
|
||||
private val brokerId: Integer = 0
|
||||
private var leaderNode: KafkaBroker = _
|
||||
|
||||
// Run tests concurrently since a throttle could be up to 1 second because quota percentage allocated is very low
|
||||
|
@ -275,61 +269,6 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
|
||||
.setTargetTimes(List(topic).asJava)
|
||||
|
||||
case ApiKeys.LEADER_AND_ISR =>
|
||||
new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Long.MaxValue,
|
||||
Seq(new LeaderAndIsrPartitionState()
|
||||
.setTopicName(tp.topic)
|
||||
.setPartitionIndex(tp.partition)
|
||||
.setControllerEpoch(Int.MaxValue)
|
||||
.setLeader(brokerId)
|
||||
.setLeaderEpoch(Int.MaxValue)
|
||||
.setIsr(List(brokerId).asJava)
|
||||
.setPartitionEpoch(2)
|
||||
.setReplicas(Seq(brokerId).asJava)
|
||||
.setIsNew(true)).asJava,
|
||||
getTopicIds().asJava,
|
||||
Set(new Node(brokerId, "localhost", 0)).asJava)
|
||||
|
||||
case ApiKeys.STOP_REPLICA =>
|
||||
val topicStates = Seq(
|
||||
new StopReplicaTopicState()
|
||||
.setTopicName(tp.topic())
|
||||
.setPartitionStates(Seq(new StopReplicaPartitionState()
|
||||
.setPartitionIndex(tp.partition())
|
||||
.setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
|
||||
.setDeletePartition(true)).asJava)
|
||||
).asJava
|
||||
new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId,
|
||||
Int.MaxValue, Long.MaxValue, false, topicStates)
|
||||
|
||||
case ApiKeys.UPDATE_METADATA =>
|
||||
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 = 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, Collections.emptyMap())
|
||||
|
||||
case ApiKeys.CONTROLLED_SHUTDOWN =>
|
||||
new ControlledShutdownRequest.Builder(
|
||||
new ControlledShutdownRequestData()
|
||||
.setBrokerId(brokerId)
|
||||
.setBrokerEpoch(Long.MaxValue),
|
||||
ApiKeys.CONTROLLED_SHUTDOWN.latestVersion)
|
||||
|
||||
case ApiKeys.OFFSET_COMMIT =>
|
||||
new OffsetCommitRequest.Builder(
|
||||
new OffsetCommitRequestData()
|
||||
|
|
|
@ -894,7 +894,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) {
|
|||
val allPartitionsMetadata = waitForAllPartitionsMetadata(brokersToValidate, topic, totalPartitionCount)
|
||||
(0 until totalPartitionCount - 1).foreach(i => {
|
||||
allPartitionsMetadata.get(new TopicPartition(topic, i)).foreach { partitionMetadata =>
|
||||
assertEquals(totalPartitionCount, partitionMetadata.replicas.size)
|
||||
assertEquals(totalPartitionCount, partitionMetadata.isr.size)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
|
|
@ -245,7 +245,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
|
|||
|
||||
private def waitForEpochChangeTo(topic: String, partition: Int, epoch: Int): Unit = {
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
brokers(0).metadataCache.getPartitionInfo(topic, partition).exists(_.leaderEpoch == epoch)
|
||||
brokers(0).metadataCache.getLeaderAndIsr(topic, partition).exists(_.leaderEpoch == epoch)
|
||||
}, "Epoch didn't change")
|
||||
}
|
||||
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.kafka.common.errors.{OperationNotAttemptedException, TopicExis
|
|||
import org.apache.kafka.common.header.Header
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.memory.MemoryPool
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.{ClientInformation, ConnectionMode, ListenerName}
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
|
@ -823,7 +822,7 @@ object TestUtils extends Logging {
|
|||
def waitForAllPartitionsMetadata[B <: KafkaBroker](
|
||||
brokers: Seq[B],
|
||||
topic: String,
|
||||
expectedNumPartitions: Int): Map[TopicPartition, UpdateMetadataPartitionState] = {
|
||||
expectedNumPartitions: Int): Map[TopicPartition, LeaderAndIsr] = {
|
||||
waitUntilTrue(
|
||||
() => brokers.forall { broker =>
|
||||
if (expectedNumPartitions == 0) {
|
||||
|
@ -836,7 +835,7 @@ object TestUtils extends Logging {
|
|||
|
||||
// since the metadata is propagated, we should get the same metadata from each server
|
||||
(0 until expectedNumPartitions).map { i =>
|
||||
new TopicPartition(topic, i) -> brokers.head.metadataCache.getPartitionInfo(topic, i).getOrElse(
|
||||
new TopicPartition(topic, i) -> brokers.head.metadataCache.getLeaderAndIsr(topic, i).getOrElse(
|
||||
throw new IllegalStateException(s"Cannot get topic: $topic, partition: $i in server metadata cache"))
|
||||
}.toMap
|
||||
}
|
||||
|
@ -853,10 +852,10 @@ object TestUtils extends Logging {
|
|||
*/
|
||||
def waitForPartitionMetadata[B <: KafkaBroker](
|
||||
brokers: Seq[B], topic: String, partition: Int,
|
||||
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): UpdateMetadataPartitionState = {
|
||||
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): LeaderAndIsr = {
|
||||
waitUntilTrue(
|
||||
() => brokers.forall { broker =>
|
||||
broker.metadataCache.getPartitionInfo(topic, partition) match {
|
||||
broker.metadataCache.getLeaderAndIsr(topic, partition) match {
|
||||
case Some(partitionState) => FetchRequest.isValidBrokerId(partitionState.leader)
|
||||
case _ => false
|
||||
}
|
||||
|
@ -864,7 +863,7 @@ object TestUtils extends Logging {
|
|||
"Partition [%s,%d] metadata not propagated after %d ms".format(topic, partition, timeout),
|
||||
waitTimeMs = timeout)
|
||||
|
||||
brokers.head.metadataCache.getPartitionInfo(topic, partition).getOrElse(
|
||||
brokers.head.metadataCache.getLeaderAndIsr(topic, partition).getOrElse(
|
||||
throw new IllegalStateException(s"Cannot get topic: $topic, partition: $partition in server metadata cache"))
|
||||
}
|
||||
|
||||
|
|
|
@ -62,6 +62,8 @@ public final class ApiMessageTypeGenerator implements TypeClassGenerator {
|
|||
String requestSchema() {
|
||||
if (requestSpec == null) {
|
||||
return "null";
|
||||
} else if (!requestSpec.hasValidVersion()) {
|
||||
return "new Schema[0]";
|
||||
} else {
|
||||
return String.format("%sData.SCHEMAS", requestSpec.name());
|
||||
}
|
||||
|
@ -70,6 +72,8 @@ public final class ApiMessageTypeGenerator implements TypeClassGenerator {
|
|||
String responseSchema() {
|
||||
if (responseSpec == null) {
|
||||
return "null";
|
||||
} else if (!requestSpec.hasValidVersion()) {
|
||||
return "new Schema[0]";
|
||||
} else {
|
||||
return String.format("%sData.SCHEMAS", responseSpec.name());
|
||||
}
|
||||
|
@ -301,12 +305,15 @@ public final class ApiMessageTypeGenerator implements TypeClassGenerator {
|
|||
buffer.printf("switch (apiKey) {%n");
|
||||
buffer.incrementIndent();
|
||||
for (Map.Entry<Short, ApiData> entry : apis.entrySet()) {
|
||||
buffer.printf("case %d:%n", entry.getKey());
|
||||
buffer.incrementIndent();
|
||||
buffer.printf("return new %s%sData();%n",
|
||||
entry.getValue().name(),
|
||||
MessageGenerator.capitalizeFirst(type));
|
||||
buffer.decrementIndent();
|
||||
MessageSpec spec = messageSpec(type, entry.getKey(), entry.getValue());
|
||||
if (spec.hasValidVersion()) {
|
||||
buffer.printf("case %d:%n", entry.getKey());
|
||||
buffer.incrementIndent();
|
||||
buffer.printf("return new %s%sData();%n",
|
||||
entry.getValue().name(),
|
||||
MessageGenerator.capitalizeFirst(type));
|
||||
buffer.decrementIndent();
|
||||
}
|
||||
}
|
||||
buffer.printf("default:%n");
|
||||
buffer.incrementIndent();
|
||||
|
@ -346,6 +353,11 @@ public final class ApiMessageTypeGenerator implements TypeClassGenerator {
|
|||
short apiKey = entry.getKey();
|
||||
ApiData apiData = entry.getValue();
|
||||
String name = apiData.name();
|
||||
|
||||
MessageSpec spec = messageSpec(type, apiKey, entry.getValue());
|
||||
if (!spec.hasValidVersion())
|
||||
continue;
|
||||
|
||||
buffer.printf("case %d: // %s%n", apiKey, MessageGenerator.capitalizeFirst(name));
|
||||
buffer.incrementIndent();
|
||||
if (type.equals("response") && apiKey == 18) {
|
||||
|
@ -355,28 +367,6 @@ public final class ApiMessageTypeGenerator implements TypeClassGenerator {
|
|||
buffer.decrementIndent();
|
||||
continue;
|
||||
}
|
||||
if (type.equals("request") && apiKey == 7) {
|
||||
buffer.printf("// Version 0 of ControlledShutdownRequest has a non-standard request header%n");
|
||||
buffer.printf("// which does not include clientId. Version 1 of ControlledShutdownRequest%n");
|
||||
buffer.printf("// and later use the standard request header.%n");
|
||||
buffer.printf("if (_version == 0) {%n");
|
||||
buffer.incrementIndent();
|
||||
buffer.printf("return (short) 0;%n");
|
||||
buffer.decrementIndent();
|
||||
buffer.printf("}%n");
|
||||
}
|
||||
ApiData data = entry.getValue();
|
||||
MessageSpec spec;
|
||||
if (type.equals("request")) {
|
||||
spec = data.requestSpec;
|
||||
} else if (type.equals("response")) {
|
||||
spec = data.responseSpec;
|
||||
} else {
|
||||
throw new RuntimeException("Invalid type " + type + " for generateHeaderVersion");
|
||||
}
|
||||
if (spec == null) {
|
||||
throw new RuntimeException("failed to find " + type + " for API key " + apiKey);
|
||||
}
|
||||
VersionConditional.forVersions(spec.flexibleVersions(),
|
||||
spec.validVersions()).
|
||||
ifMember(__ -> {
|
||||
|
@ -407,6 +397,20 @@ public final class ApiMessageTypeGenerator implements TypeClassGenerator {
|
|||
buffer.printf("}%n");
|
||||
}
|
||||
|
||||
private static MessageSpec messageSpec(String type, short apiKey, ApiData apiData) {
|
||||
MessageSpec spec;
|
||||
if (type.equals("request")) {
|
||||
spec = apiData.requestSpec;
|
||||
} else if (type.equals("response")) {
|
||||
spec = apiData.responseSpec;
|
||||
} else {
|
||||
throw new RuntimeException("Invalid type " + type);
|
||||
}
|
||||
if (spec == null)
|
||||
throw new RuntimeException("failed to find " + type + " for API key " + apiKey);
|
||||
return spec;
|
||||
}
|
||||
|
||||
private void generateListenerTypesEnum() {
|
||||
buffer.printf("public enum ListenerType {%n");
|
||||
buffer.incrementIndent();
|
||||
|
|
|
@ -29,6 +29,7 @@ import net.sourceforge.argparse4j.inf.ArgumentParser;
|
|||
import net.sourceforge.argparse4j.inf.Namespace;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.DirectoryStream;
|
||||
import java.nio.file.Files;
|
||||
|
@ -39,6 +40,7 @@ import java.util.Collections;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
|
||||
import static net.sourceforge.argparse4j.impl.Arguments.store;
|
||||
|
||||
|
@ -237,25 +239,14 @@ public final class MessageGenerator {
|
|||
Files.createDirectories(Paths.get(outputDir));
|
||||
int numProcessed = 0;
|
||||
|
||||
List<TypeClassGenerator> typeClassGenerators =
|
||||
createTypeClassGenerators(packageName, typeClassGeneratorTypes);
|
||||
HashSet<String> outputFileNames = new HashSet<>();
|
||||
try (DirectoryStream<Path> directoryStream = Files
|
||||
.newDirectoryStream(Paths.get(inputDir), JSON_GLOB)) {
|
||||
List<TypeClassGenerator> typeClassGenerators = createTypeClassGenerators(packageName, typeClassGeneratorTypes);
|
||||
Set<String> outputFileNames = new HashSet<>();
|
||||
try (DirectoryStream<Path> directoryStream = Files.newDirectoryStream(Paths.get(inputDir), JSON_GLOB)) {
|
||||
for (Path inputPath : directoryStream) {
|
||||
try {
|
||||
MessageSpec spec = JSON_SERDE.
|
||||
readValue(inputPath.toFile(), MessageSpec.class);
|
||||
List<MessageClassGenerator> generators =
|
||||
createMessageClassGenerators(packageName, messageClassGeneratorTypes);
|
||||
for (MessageClassGenerator generator : generators) {
|
||||
String name = generator.outputName(spec) + JAVA_SUFFIX;
|
||||
outputFileNames.add(name);
|
||||
Path outputPath = Paths.get(outputDir, name);
|
||||
try (BufferedWriter writer = Files.newBufferedWriter(outputPath, StandardCharsets.UTF_8)) {
|
||||
generator.generateAndWrite(spec, writer);
|
||||
}
|
||||
}
|
||||
MessageSpec spec = JSON_SERDE.readValue(inputPath.toFile(), MessageSpec.class);
|
||||
outputFileNames.addAll(
|
||||
generateAndWriteMessageClasses(spec, packageName, outputDir, messageClassGeneratorTypes));
|
||||
numProcessed++;
|
||||
typeClassGenerators.forEach(generator -> generator.registerMessageType(spec));
|
||||
} catch (Exception e) {
|
||||
|
@ -265,13 +256,10 @@ public final class MessageGenerator {
|
|||
}
|
||||
for (TypeClassGenerator typeClassGenerator : typeClassGenerators) {
|
||||
outputFileNames.add(typeClassGenerator.outputName());
|
||||
Path factoryOutputPath = Paths.get(outputDir, typeClassGenerator.outputName());
|
||||
try (BufferedWriter writer = Files.newBufferedWriter(factoryOutputPath, StandardCharsets.UTF_8)) {
|
||||
typeClassGenerator.generateAndWrite(writer);
|
||||
}
|
||||
generateAndWriteTypeClasses(outputDir, typeClassGenerator);
|
||||
}
|
||||
try (DirectoryStream<Path> directoryStream = Files.
|
||||
newDirectoryStream(Paths.get(outputDir))) {
|
||||
|
||||
try (DirectoryStream<Path> directoryStream = Files.newDirectoryStream(Paths.get(outputDir))) {
|
||||
for (Path outputPath : directoryStream) {
|
||||
Path fileName = outputPath.getFileName();
|
||||
if (fileName != null) {
|
||||
|
@ -284,6 +272,37 @@ public final class MessageGenerator {
|
|||
System.out.printf("MessageGenerator: processed %d Kafka message JSON files(s).%n", numProcessed);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate and write message classes.
|
||||
* @return output file names.
|
||||
*/
|
||||
static Set<String> generateAndWriteMessageClasses(MessageSpec spec,
|
||||
String packageName,
|
||||
String outputDir,
|
||||
List<String> messageClassGeneratorTypes) throws Exception {
|
||||
var outputFileNames = new HashSet<String>();
|
||||
// Only generate `Data` classes for apis that have valid version(s)
|
||||
if (spec.hasValidVersion()) {
|
||||
List<MessageClassGenerator> generators = createMessageClassGenerators(packageName, messageClassGeneratorTypes);
|
||||
for (MessageClassGenerator generator : generators) {
|
||||
String name = generator.outputName(spec) + JAVA_SUFFIX;
|
||||
outputFileNames.add(name);
|
||||
Path outputPath = Paths.get(outputDir, name);
|
||||
try (BufferedWriter writer = Files.newBufferedWriter(outputPath, StandardCharsets.UTF_8)) {
|
||||
generator.generateAndWrite(spec, writer);
|
||||
}
|
||||
}
|
||||
}
|
||||
return outputFileNames;
|
||||
}
|
||||
|
||||
static void generateAndWriteTypeClasses(String outputDir, TypeClassGenerator typeClassGenerator) throws IOException {
|
||||
Path factoryOutputPath = Paths.get(outputDir, typeClassGenerator.outputName());
|
||||
try (BufferedWriter writer = Files.newBufferedWriter(factoryOutputPath, StandardCharsets.UTF_8)) {
|
||||
typeClassGenerator.generateAndWrite(writer);
|
||||
}
|
||||
}
|
||||
|
||||
public static String capitalizeFirst(String string) {
|
||||
if (string.isEmpty()) {
|
||||
return string;
|
||||
|
|
|
@ -58,45 +58,54 @@ public final class MessageSpec {
|
|||
this.type = Objects.requireNonNull(type);
|
||||
this.commonStructs = commonStructs == null ? Collections.emptyList() :
|
||||
List.copyOf(commonStructs);
|
||||
if (flexibleVersions == null) {
|
||||
throw new RuntimeException("You must specify a value for flexibleVersions. " +
|
||||
"Please use 0+ for all new messages.");
|
||||
}
|
||||
this.flexibleVersions = Versions.parse(flexibleVersions, Versions.NONE);
|
||||
if ((!this.flexibleVersions().empty()) &&
|
||||
(this.flexibleVersions.highest() < Short.MAX_VALUE)) {
|
||||
throw new RuntimeException("Field " + name + " specifies flexibleVersions " +
|
||||
this.flexibleVersions + ", which is not open-ended. flexibleVersions must " +
|
||||
"be either none, or an open-ended range (that ends with a plus sign).");
|
||||
}
|
||||
|
||||
if (listeners != null && !listeners.isEmpty() && type != MessageSpecType.REQUEST) {
|
||||
throw new RuntimeException("The `requestScope` property is only valid for " +
|
||||
"messages with type `request`");
|
||||
}
|
||||
this.listeners = listeners;
|
||||
|
||||
if (latestVersionUnstable && type != MessageSpecType.REQUEST) {
|
||||
throw new RuntimeException("The `latestVersionUnstable` property is only valid for " +
|
||||
"messages with type `request`");
|
||||
}
|
||||
this.latestVersionUnstable = latestVersionUnstable;
|
||||
|
||||
if (type == MessageSpecType.COORDINATOR_KEY) {
|
||||
if (this.apiKey.isEmpty()) {
|
||||
throw new RuntimeException("The ApiKey must be set for messages " + name + " with type `coordinator-key`");
|
||||
// If the struct has no valid versions (the typical use case is to completely remove support for
|
||||
// an existing protocol api while ensuring the api key id is not reused), we configure the spec
|
||||
// to effectively be empty
|
||||
if (struct.versions().empty()) {
|
||||
this.flexibleVersions = Versions.NONE;
|
||||
this.listeners = Collections.emptyList();
|
||||
this.latestVersionUnstable = false;
|
||||
} else {
|
||||
if (flexibleVersions == null) {
|
||||
throw new RuntimeException("You must specify a value for flexibleVersions. " +
|
||||
"Please use 0+ for all new messages.");
|
||||
}
|
||||
if (!this.validVersions().equals(new Versions((short) 0, ((short) 0)))) {
|
||||
throw new RuntimeException("The Versions must be set to `0` for messages " + name + " with type `coordinator-key`");
|
||||
this.flexibleVersions = Versions.parse(flexibleVersions, Versions.NONE);
|
||||
if ((!this.flexibleVersions().empty()) &&
|
||||
(this.flexibleVersions.highest() < Short.MAX_VALUE)) {
|
||||
throw new RuntimeException("Field " + name + " specifies flexibleVersions " +
|
||||
this.flexibleVersions + ", which is not open-ended. flexibleVersions must " +
|
||||
"be either none, or an open-ended range (that ends with a plus sign).");
|
||||
}
|
||||
if (!this.flexibleVersions.empty()) {
|
||||
throw new RuntimeException("The FlexibleVersions are not supported for messages " + name + " with type `coordinator-key`");
|
||||
}
|
||||
}
|
||||
|
||||
if (type == MessageSpecType.COORDINATOR_VALUE) {
|
||||
if (this.apiKey.isEmpty()) {
|
||||
throw new RuntimeException("The ApiKey must be set for messages with type `coordinator-value`");
|
||||
if (listeners != null && !listeners.isEmpty() && type != MessageSpecType.REQUEST) {
|
||||
throw new RuntimeException("The `requestScope` property is only valid for " +
|
||||
"messages with type `request`");
|
||||
}
|
||||
this.listeners = listeners;
|
||||
|
||||
if (latestVersionUnstable && type != MessageSpecType.REQUEST) {
|
||||
throw new RuntimeException("The `latestVersionUnstable` property is only valid for " +
|
||||
"messages with type `request`");
|
||||
}
|
||||
this.latestVersionUnstable = latestVersionUnstable;
|
||||
|
||||
if (type == MessageSpecType.COORDINATOR_KEY) {
|
||||
if (this.apiKey.isEmpty()) {
|
||||
throw new RuntimeException("The ApiKey must be set for messages " + name + " with type `coordinator-key`");
|
||||
}
|
||||
if (!this.validVersions().equals(new Versions((short) 0, ((short) 0)))) {
|
||||
throw new RuntimeException("The Versions must be set to `0` for messages " + name + " with type `coordinator-key`");
|
||||
}
|
||||
if (!this.flexibleVersions.empty()) {
|
||||
throw new RuntimeException("The FlexibleVersions are not supported for messages " + name + " with type `coordinator-key`");
|
||||
}
|
||||
}
|
||||
|
||||
if (type == MessageSpecType.COORDINATOR_VALUE) {
|
||||
if (this.apiKey.isEmpty()) {
|
||||
throw new RuntimeException("The ApiKey must be set for messages with type `coordinator-value`");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -110,6 +119,10 @@ public final class MessageSpec {
|
|||
return struct.name();
|
||||
}
|
||||
|
||||
public boolean hasValidVersion() {
|
||||
return !struct.versions().empty();
|
||||
}
|
||||
|
||||
public Versions validVersions() {
|
||||
return struct.versions();
|
||||
}
|
||||
|
|
|
@ -19,6 +19,14 @@ package org.apache.kafka.message;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -69,4 +77,70 @@ public class MessageGeneratorTest {
|
|||
assertEquals(MessageGenerator.UNSIGNED_SHORT_MAX, 0xFFFF);
|
||||
assertEquals(MessageGenerator.UNSIGNED_INT_MAX, 0xFFFFFFFFL);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGenerateAndWriteMessageClasses(@TempDir Path tempDir) throws Exception {
|
||||
var generatorTypes = List.of("MessageDataGenerator", "JsonConverterGenerator");
|
||||
|
||||
MessageSpec testRequestSpec = MessageGenerator.JSON_SERDE.readValue(String.join("", List.of(
|
||||
"{",
|
||||
" \"apiKey\": 0,",
|
||||
" \"type\": \"request\",",
|
||||
" \"name\": \"FooBarRequest\",",
|
||||
" \"validVersions\": \"none\"",
|
||||
"}")), MessageSpec.class);
|
||||
MessageSpec testResponseSpec = MessageGenerator.JSON_SERDE.readValue(String.join("", List.of(
|
||||
"{",
|
||||
" \"apiKey\": 0,",
|
||||
" \"type\": \"response\",",
|
||||
" \"name\": \"FooBarRespose\",",
|
||||
" \"validVersions\": \"none\"",
|
||||
"}")), MessageSpec.class);
|
||||
|
||||
var outputFiles = MessageGenerator.generateAndWriteMessageClasses(testRequestSpec, "kafka",
|
||||
tempDir.toAbsolutePath().toString(), generatorTypes);
|
||||
assertEquals(Set.of(), outputFiles);
|
||||
outputFiles = MessageGenerator.generateAndWriteMessageClasses(testResponseSpec, "kafka",
|
||||
tempDir.toAbsolutePath().toString(), generatorTypes);
|
||||
assertEquals(Set.of(), outputFiles);
|
||||
var typeGenerator = new ApiMessageTypeGenerator("kafka");
|
||||
typeGenerator.registerMessageType(testRequestSpec);
|
||||
typeGenerator.registerMessageType(testResponseSpec);
|
||||
typeGenerator.generateAndWrite(new BufferedWriter(new StringWriter()));
|
||||
|
||||
testRequestSpec = MessageGenerator.JSON_SERDE.readValue(String.join("", Arrays.asList(
|
||||
"{",
|
||||
" \"apiKey\": 0,",
|
||||
" \"type\": \"request\",",
|
||||
" \"name\": \"FooBarRequest\",",
|
||||
" \"validVersions\": \"0-2\",",
|
||||
" \"flexibleVersions\": \"none\",",
|
||||
" \"fields\": [",
|
||||
" { \"name\": \"field1\", \"type\": \"int32\", \"versions\": \"0+\" }",
|
||||
" ]",
|
||||
"}")), MessageSpec.class);
|
||||
testResponseSpec = MessageGenerator.JSON_SERDE.readValue(String.join("", Arrays.asList(
|
||||
"{",
|
||||
" \"apiKey\": 0,",
|
||||
" \"type\": \"response\",",
|
||||
" \"name\": \"FooBarResponse\",",
|
||||
" \"validVersions\": \"0-2\",",
|
||||
" \"flexibleVersions\": \"none\",",
|
||||
" \"fields\": [",
|
||||
" { \"name\": \"field1\", \"type\": \"int32\", \"versions\": \"0+\" }",
|
||||
" ]",
|
||||
"}")), MessageSpec.class);
|
||||
|
||||
outputFiles = MessageGenerator.generateAndWriteMessageClasses(testRequestSpec, "kafka",
|
||||
tempDir.toAbsolutePath().toString(), generatorTypes);
|
||||
assertEquals(Set.of("FooBarRequestDataJsonConverter.java", "FooBarRequestData.java"), outputFiles);
|
||||
outputFiles = MessageGenerator.generateAndWriteMessageClasses(testResponseSpec, "kafka",
|
||||
tempDir.toAbsolutePath().toString(), generatorTypes);
|
||||
assertEquals(Set.of("FooBarResponseDataJsonConverter.java", "FooBarResponseData.java"), outputFiles);
|
||||
typeGenerator = new ApiMessageTypeGenerator("kafka");
|
||||
typeGenerator.registerMessageType(testRequestSpec);
|
||||
typeGenerator.registerMessageType(testResponseSpec);
|
||||
typeGenerator.generateAndWrite(new BufferedWriter(new StringWriter()));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.junit.jupiter.api.Timeout;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -150,4 +151,21 @@ public class StructRegistryTest {
|
|||
assertEquals(structRegistry.findStruct(field2).name(), "TestInlineStruct");
|
||||
assertFalse(structRegistry.isStructArrayWithKeys(field2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidVersionsIsNone() throws Exception {
|
||||
MessageSpec testMessageSpec = MessageGenerator.JSON_SERDE.readValue(String.join("", List.of(
|
||||
"{",
|
||||
" \"type\": \"request\",",
|
||||
" \"name\": \"FooBar\",",
|
||||
" \"validVersions\": \"none\"",
|
||||
"}")), MessageSpec.class);
|
||||
StructRegistry structRegistry = new StructRegistry();
|
||||
structRegistry.register(testMessageSpec);
|
||||
|
||||
assertFalse(testMessageSpec.hasValidVersion());
|
||||
assertEquals(List.of(), testMessageSpec.fields());
|
||||
assertFalse(structRegistry.structs().hasNext());
|
||||
assertFalse(structRegistry.commonStructs().hasNext());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ public class VersionsTest {
|
|||
|
||||
@Test
|
||||
public void testVersionsParse() {
|
||||
assertEquals(Versions.NONE, Versions.parse(" none ", null));
|
||||
assertEquals(Versions.NONE, Versions.parse(null, Versions.NONE));
|
||||
assertEquals(Versions.ALL, Versions.parse(" ", Versions.ALL));
|
||||
assertEquals(Versions.ALL, Versions.parse("", Versions.ALL));
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.kafka.common.TopicIdPartition;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.FetchResponseData;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData;
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
|
@ -51,6 +50,7 @@ import org.apache.kafka.common.record.BaseRecords;
|
|||
import org.apache.kafka.common.record.RecordsSend;
|
||||
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.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
@ -167,7 +167,7 @@ public class ReplicaFetcherThreadBenchmark {
|
|||
TopicPartition tp = new TopicPartition("topic", i);
|
||||
|
||||
List<Integer> replicas = Arrays.asList(0, 1, 2);
|
||||
LeaderAndIsrRequestData.LeaderAndIsrPartitionState partitionState = new LeaderAndIsrRequestData.LeaderAndIsrPartitionState()
|
||||
LeaderAndIsrRequest.PartitionState partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(0)
|
||||
.setLeaderEpoch(0)
|
||||
|
|
|
@ -40,7 +40,6 @@ import kafka.server.share.SharePartitionManager;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.memory.MemoryPool;
|
||||
import org.apache.kafka.common.message.ApiMessageType;
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint;
|
||||
import org.apache.kafka.common.metadata.PartitionRecord;
|
||||
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
|
||||
import org.apache.kafka.common.metadata.TopicRecord;
|
||||
|
@ -141,12 +140,7 @@ public class KRaftMetadataRequestBenchmark {
|
|||
MetadataDelta buildupMetadataDelta = new MetadataDelta(MetadataImage.EMPTY);
|
||||
IntStream.range(0, 5).forEach(brokerId -> {
|
||||
RegisterBrokerRecord.BrokerEndpointCollection endpoints = new RegisterBrokerRecord.BrokerEndpointCollection();
|
||||
endpoints(brokerId).forEach(endpoint ->
|
||||
endpoints.add(new RegisterBrokerRecord.BrokerEndpoint().
|
||||
setHost(endpoint.host()).
|
||||
setPort(endpoint.port()).
|
||||
setName(endpoint.listener()).
|
||||
setSecurityProtocol(endpoint.securityProtocol())));
|
||||
endpoints(brokerId).forEach(endpoint -> endpoints.add(endpoint));
|
||||
buildupMetadataDelta.replay(new RegisterBrokerRecord().
|
||||
setBrokerId(brokerId).
|
||||
setBrokerEpoch(100L).
|
||||
|
@ -172,13 +166,13 @@ public class KRaftMetadataRequestBenchmark {
|
|||
metadataCache.setImage(buildupMetadataDelta.apply(MetadataProvenance.EMPTY));
|
||||
}
|
||||
|
||||
private List<UpdateMetadataEndpoint> endpoints(final int brokerId) {
|
||||
private List<RegisterBrokerRecord.BrokerEndpoint> endpoints(final int brokerId) {
|
||||
return Collections.singletonList(
|
||||
new UpdateMetadataEndpoint()
|
||||
.setHost("host_" + brokerId)
|
||||
.setPort(9092)
|
||||
.setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)
|
||||
.setListener(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT).value()));
|
||||
new RegisterBrokerRecord.BrokerEndpoint().
|
||||
setHost("host_" + brokerId).
|
||||
setPort(9092).
|
||||
setName(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT).value()).
|
||||
setSecurityProtocol(SecurityProtocol.PLAINTEXT.id));
|
||||
}
|
||||
|
||||
private KafkaApis createKafkaApis() {
|
||||
|
|
|
@ -29,9 +29,9 @@ import kafka.server.metadata.MockConfigRepository;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.compress.Compression;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData;
|
||||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.SimpleRecord;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
|
@ -152,7 +152,7 @@ public class PartitionMakeFollowerBenchmark {
|
|||
|
||||
@Benchmark
|
||||
public boolean testMakeFollower() {
|
||||
LeaderAndIsrRequestData.LeaderAndIsrPartitionState partitionState = new LeaderAndIsrRequestData.LeaderAndIsrPartitionState()
|
||||
LeaderAndIsrRequest.PartitionState partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(0)
|
||||
.setLeaderEpoch(0)
|
||||
|
|
|
@ -29,7 +29,7 @@ import kafka.server.metadata.MockConfigRepository;
|
|||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.apache.kafka.server.util.KafkaScheduler;
|
||||
|
@ -116,7 +116,7 @@ public class UpdateFollowerFetchStateBenchmark {
|
|||
replicas.add(0);
|
||||
replicas.add(1);
|
||||
replicas.add(2);
|
||||
LeaderAndIsrPartitionState partitionState = new LeaderAndIsrPartitionState()
|
||||
LeaderAndIsrRequest.PartitionState partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(0)
|
||||
.setLeaderEpoch(0)
|
||||
|
|
|
@ -31,8 +31,8 @@ import kafka.utils.TestUtils;
|
|||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
|
@ -199,7 +199,7 @@ public class PartitionCreationBench {
|
|||
inSync.add(1);
|
||||
inSync.add(2);
|
||||
|
||||
LeaderAndIsrRequestData.LeaderAndIsrPartitionState partitionState = new LeaderAndIsrRequestData.LeaderAndIsrPartitionState()
|
||||
LeaderAndIsrRequest.PartitionState partitionState = new LeaderAndIsrRequest.PartitionState()
|
||||
.setControllerEpoch(0)
|
||||
.setLeader(0)
|
||||
.setLeaderEpoch(0)
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.kafka.common.DirectoryId;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.errors.InvalidReplicaDirectoriesException;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
|
||||
import org.apache.kafka.common.metadata.PartitionChangeRecord;
|
||||
import org.apache.kafka.common.metadata.PartitionRecord;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
|
||||
import org.apache.kafka.image.writer.ImageWriterOptions;
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||
|
||||
|
@ -406,9 +406,9 @@ public class PartitionRegistration {
|
|||
return new ApiMessageAndVersion(record, options.metadataVersion().partitionRecordVersion());
|
||||
}
|
||||
|
||||
public LeaderAndIsrPartitionState toLeaderAndIsrPartitionState(TopicPartition tp,
|
||||
boolean isNew) {
|
||||
return new LeaderAndIsrPartitionState().
|
||||
public LeaderAndIsrRequest.PartitionState toLeaderAndIsrPartitionState(TopicPartition tp,
|
||||
boolean isNew) {
|
||||
return new LeaderAndIsrRequest.PartitionState().
|
||||
setTopicName(tp.topic()).
|
||||
setPartitionIndex(tp.partition()).
|
||||
setControllerEpoch(-1).
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.kafka.metadata;
|
|||
import org.apache.kafka.common.DirectoryId;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
|
||||
import org.apache.kafka.common.metadata.PartitionChangeRecord;
|
||||
import org.apache.kafka.common.metadata.PartitionRecord;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
|
||||
import org.apache.kafka.image.writer.ImageWriterOptions;
|
||||
import org.apache.kafka.image.writer.UnwritableMetadataException;
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||
|
@ -116,7 +116,7 @@ public class PartitionRegistrationTest {
|
|||
Uuid.fromString("bAAlGAz1TN2doZjtWlvhRQ")
|
||||
}).
|
||||
setIsr(new int[]{2, 3, 4}).setLeader(2).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(234).setPartitionEpoch(567).build();
|
||||
assertEquals(new LeaderAndIsrPartitionState().
|
||||
assertEquals(new LeaderAndIsrRequest.PartitionState().
|
||||
setTopicName("foo").
|
||||
setPartitionIndex(1).
|
||||
setControllerEpoch(-1).
|
||||
|
@ -129,7 +129,7 @@ public class PartitionRegistrationTest {
|
|||
setRemovingReplicas(Collections.emptyList()).
|
||||
setIsNew(true).toString(),
|
||||
a.toLeaderAndIsrPartitionState(new TopicPartition("foo", 1), true).toString());
|
||||
assertEquals(new LeaderAndIsrPartitionState().
|
||||
assertEquals(new LeaderAndIsrRequest.PartitionState().
|
||||
setTopicName("bar").
|
||||
setPartitionIndex(0).
|
||||
setControllerEpoch(-1).
|
||||
|
|
|
@ -50,8 +50,6 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeRequestDataJsonConve
|
|||
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ControlledShutdownRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ControlledShutdownResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ControllerRegistrationRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ControllerRegistrationResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.CreateAclsRequestDataJsonConverter;
|
||||
|
@ -126,8 +124,6 @@ import org.apache.kafka.common.message.InitializeShareGroupStateRequestDataJsonC
|
|||
import org.apache.kafka.common.message.InitializeShareGroupStateResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.JoinGroupRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.JoinGroupResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.LeaveGroupRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.LeaveGroupResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ListClientMetricsResourcesRequestDataJsonConverter;
|
||||
|
@ -175,8 +171,6 @@ import org.apache.kafka.common.message.ShareGroupDescribeRequestDataJsonConverte
|
|||
import org.apache.kafka.common.message.ShareGroupDescribeResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ShareGroupHeartbeatRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.ShareGroupHeartbeatResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.StopReplicaRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.StopReplicaResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.StreamsGroupDescribeRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.StreamsGroupDescribeResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestDataJsonConverter;
|
||||
|
@ -189,8 +183,6 @@ import org.apache.kafka.common.message.UnregisterBrokerRequestDataJsonConverter;
|
|||
import org.apache.kafka.common.message.UnregisterBrokerResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.UpdateFeaturesRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.UpdateFeaturesResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.UpdateMetadataResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.UpdateRaftVoterRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.UpdateRaftVoterResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.VoteRequestDataJsonConverter;
|
||||
|
@ -236,8 +228,6 @@ import org.apache.kafka.common.requests.ConsumerGroupDescribeRequest;
|
|||
import org.apache.kafka.common.requests.ConsumerGroupDescribeResponse;
|
||||
import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest;
|
||||
import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse;
|
||||
import org.apache.kafka.common.requests.ControlledShutdownRequest;
|
||||
import org.apache.kafka.common.requests.ControlledShutdownResponse;
|
||||
import org.apache.kafka.common.requests.ControllerRegistrationRequest;
|
||||
import org.apache.kafka.common.requests.ControllerRegistrationResponse;
|
||||
import org.apache.kafka.common.requests.CreateAclsRequest;
|
||||
|
@ -312,8 +302,6 @@ import org.apache.kafka.common.requests.InitializeShareGroupStateRequest;
|
|||
import org.apache.kafka.common.requests.InitializeShareGroupStateResponse;
|
||||
import org.apache.kafka.common.requests.JoinGroupRequest;
|
||||
import org.apache.kafka.common.requests.JoinGroupResponse;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest;
|
||||
import org.apache.kafka.common.requests.LeaderAndIsrResponse;
|
||||
import org.apache.kafka.common.requests.LeaveGroupRequest;
|
||||
import org.apache.kafka.common.requests.LeaveGroupResponse;
|
||||
import org.apache.kafka.common.requests.ListClientMetricsResourcesRequest;
|
||||
|
@ -362,8 +350,6 @@ import org.apache.kafka.common.requests.ShareGroupDescribeRequest;
|
|||
import org.apache.kafka.common.requests.ShareGroupDescribeResponse;
|
||||
import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest;
|
||||
import org.apache.kafka.common.requests.ShareGroupHeartbeatResponse;
|
||||
import org.apache.kafka.common.requests.StopReplicaRequest;
|
||||
import org.apache.kafka.common.requests.StopReplicaResponse;
|
||||
import org.apache.kafka.common.requests.StreamsGroupDescribeRequest;
|
||||
import org.apache.kafka.common.requests.StreamsGroupDescribeResponse;
|
||||
import org.apache.kafka.common.requests.StreamsGroupHeartbeatRequest;
|
||||
|
@ -376,8 +362,6 @@ import org.apache.kafka.common.requests.UnregisterBrokerRequest;
|
|||
import org.apache.kafka.common.requests.UnregisterBrokerResponse;
|
||||
import org.apache.kafka.common.requests.UpdateFeaturesRequest;
|
||||
import org.apache.kafka.common.requests.UpdateFeaturesResponse;
|
||||
import org.apache.kafka.common.requests.UpdateMetadataRequest;
|
||||
import org.apache.kafka.common.requests.UpdateMetadataResponse;
|
||||
import org.apache.kafka.common.requests.UpdateRaftVoterRequest;
|
||||
import org.apache.kafka.common.requests.UpdateRaftVoterResponse;
|
||||
import org.apache.kafka.common.requests.VoteRequest;
|
||||
|
@ -435,8 +419,6 @@ public class RequestConvertToJson {
|
|||
return ConsumerGroupDescribeRequestDataJsonConverter.write(((ConsumerGroupDescribeRequest) request).data(), request.version());
|
||||
case CONSUMER_GROUP_HEARTBEAT:
|
||||
return ConsumerGroupHeartbeatRequestDataJsonConverter.write(((ConsumerGroupHeartbeatRequest) request).data(), request.version());
|
||||
case CONTROLLED_SHUTDOWN:
|
||||
return ControlledShutdownRequestDataJsonConverter.write(((ControlledShutdownRequest) request).data(), request.version());
|
||||
case CONTROLLER_REGISTRATION:
|
||||
return ControllerRegistrationRequestDataJsonConverter.write(((ControllerRegistrationRequest) request).data(), request.version());
|
||||
case CREATE_ACLS:
|
||||
|
@ -511,8 +493,6 @@ public class RequestConvertToJson {
|
|||
return InitProducerIdRequestDataJsonConverter.write(((InitProducerIdRequest) request).data(), request.version());
|
||||
case JOIN_GROUP:
|
||||
return JoinGroupRequestDataJsonConverter.write(((JoinGroupRequest) request).data(), request.version());
|
||||
case LEADER_AND_ISR:
|
||||
return LeaderAndIsrRequestDataJsonConverter.write(((LeaderAndIsrRequest) request).data(), request.version());
|
||||
case LEAVE_GROUP:
|
||||
return LeaveGroupRequestDataJsonConverter.write(((LeaveGroupRequest) request).data(), request.version());
|
||||
case LIST_CLIENT_METRICS_RESOURCES:
|
||||
|
@ -563,8 +543,6 @@ public class RequestConvertToJson {
|
|||
return StreamsGroupDescribeRequestDataJsonConverter.write(((StreamsGroupDescribeRequest) request).data(), request.version());
|
||||
case STREAMS_GROUP_HEARTBEAT:
|
||||
return StreamsGroupHeartbeatRequestDataJsonConverter.write(((StreamsGroupHeartbeatRequest) request).data(), request.version());
|
||||
case STOP_REPLICA:
|
||||
return StopReplicaRequestDataJsonConverter.write(((StopReplicaRequest) request).data(), request.version());
|
||||
case SYNC_GROUP:
|
||||
return SyncGroupRequestDataJsonConverter.write(((SyncGroupRequest) request).data(), request.version());
|
||||
case TXN_OFFSET_COMMIT:
|
||||
|
@ -573,8 +551,6 @@ public class RequestConvertToJson {
|
|||
return UnregisterBrokerRequestDataJsonConverter.write(((UnregisterBrokerRequest) request).data(), request.version());
|
||||
case UPDATE_FEATURES:
|
||||
return UpdateFeaturesRequestDataJsonConverter.write(((UpdateFeaturesRequest) request).data(), request.version());
|
||||
case UPDATE_METADATA:
|
||||
return UpdateMetadataRequestDataJsonConverter.write(((UpdateMetadataRequest) request).data(), request.version());
|
||||
case UPDATE_RAFT_VOTER:
|
||||
return UpdateRaftVoterRequestDataJsonConverter.write(((UpdateRaftVoterRequest) request).data(), request.version());
|
||||
case VOTE:
|
||||
|
@ -625,8 +601,6 @@ public class RequestConvertToJson {
|
|||
return ConsumerGroupDescribeResponseDataJsonConverter.write(((ConsumerGroupDescribeResponse) response).data(), version);
|
||||
case CONSUMER_GROUP_HEARTBEAT:
|
||||
return ConsumerGroupHeartbeatResponseDataJsonConverter.write(((ConsumerGroupHeartbeatResponse) response).data(), version);
|
||||
case CONTROLLED_SHUTDOWN:
|
||||
return ControlledShutdownResponseDataJsonConverter.write(((ControlledShutdownResponse) response).data(), version);
|
||||
case CONTROLLER_REGISTRATION:
|
||||
return ControllerRegistrationResponseDataJsonConverter.write(((ControllerRegistrationResponse) response).data(), version);
|
||||
case CREATE_ACLS:
|
||||
|
@ -701,8 +675,6 @@ public class RequestConvertToJson {
|
|||
return InitProducerIdResponseDataJsonConverter.write(((InitProducerIdResponse) response).data(), version);
|
||||
case JOIN_GROUP:
|
||||
return JoinGroupResponseDataJsonConverter.write(((JoinGroupResponse) response).data(), version);
|
||||
case LEADER_AND_ISR:
|
||||
return LeaderAndIsrResponseDataJsonConverter.write(((LeaderAndIsrResponse) response).data(), version);
|
||||
case LEAVE_GROUP:
|
||||
return LeaveGroupResponseDataJsonConverter.write(((LeaveGroupResponse) response).data(), version);
|
||||
case LIST_CLIENT_METRICS_RESOURCES:
|
||||
|
@ -753,8 +725,6 @@ public class RequestConvertToJson {
|
|||
return StreamsGroupDescribeResponseDataJsonConverter.write(((StreamsGroupDescribeResponse) response).data(), version);
|
||||
case STREAMS_GROUP_HEARTBEAT:
|
||||
return StreamsGroupHeartbeatResponseDataJsonConverter.write(((StreamsGroupHeartbeatResponse) response).data(), version);
|
||||
case STOP_REPLICA:
|
||||
return StopReplicaResponseDataJsonConverter.write(((StopReplicaResponse) response).data(), version);
|
||||
case SYNC_GROUP:
|
||||
return SyncGroupResponseDataJsonConverter.write(((SyncGroupResponse) response).data(), version);
|
||||
case TXN_OFFSET_COMMIT:
|
||||
|
@ -763,8 +733,6 @@ public class RequestConvertToJson {
|
|||
return UnregisterBrokerResponseDataJsonConverter.write(((UnregisterBrokerResponse) response).data(), version);
|
||||
case UPDATE_FEATURES:
|
||||
return UpdateFeaturesResponseDataJsonConverter.write(((UpdateFeaturesResponse) response).data(), version);
|
||||
case UPDATE_METADATA:
|
||||
return UpdateMetadataResponseDataJsonConverter.write(((UpdateMetadataResponse) response).data(), version);
|
||||
case UPDATE_RAFT_VOTER:
|
||||
return UpdateRaftVoterResponseDataJsonConverter.write(((UpdateRaftVoterResponse) response).data(), version);
|
||||
case VOTE:
|
||||
|
|
|
@ -47,24 +47,27 @@ public class RequestConvertToJsonTest {
|
|||
public void testAllRequestTypesHandled() {
|
||||
List<String> unhandledKeys = new ArrayList<>();
|
||||
for (ApiKeys key : ApiKeys.values()) {
|
||||
short version = key.latestVersion();
|
||||
ApiMessage message;
|
||||
if (key == ApiKeys.DESCRIBE_ACLS) {
|
||||
message = ApiMessageType.fromApiKey(key.id).newRequest();
|
||||
DescribeAclsRequestData requestData = (DescribeAclsRequestData) message;
|
||||
requestData.setPatternTypeFilter((byte) 1);
|
||||
requestData.setResourceTypeFilter((byte) 1);
|
||||
requestData.setPermissionType((byte) 1);
|
||||
requestData.setOperation((byte) 1);
|
||||
} else {
|
||||
message = ApiMessageType.fromApiKey(key.id).newRequest();
|
||||
}
|
||||
ByteBuffer bytes = MessageUtil.toByteBuffer(message, version);
|
||||
AbstractRequest req = AbstractRequest.parseRequest(key, version, bytes).request;
|
||||
try {
|
||||
RequestConvertToJson.request(req);
|
||||
} catch (IllegalStateException e) {
|
||||
unhandledKeys.add(key.toString());
|
||||
if (key.hasValidVersion()) {
|
||||
short version = key.latestVersion();
|
||||
ApiMessage message;
|
||||
if (key == ApiKeys.DESCRIBE_ACLS) {
|
||||
message = ApiMessageType.fromApiKey(key.id).newRequest();
|
||||
DescribeAclsRequestData requestData = (DescribeAclsRequestData) message;
|
||||
requestData.setPatternTypeFilter((byte) 1);
|
||||
requestData.setResourceTypeFilter((byte) 1);
|
||||
requestData.setPermissionType((byte) 1);
|
||||
requestData.setOperation((byte) 1);
|
||||
} else {
|
||||
message = ApiMessageType.fromApiKey(key.id).newRequest();
|
||||
}
|
||||
ByteBuffer bytes = MessageUtil.toByteBuffer(message, version);
|
||||
AbstractRequest req = AbstractRequest.parseRequest(key, version, bytes).request;
|
||||
try {
|
||||
RequestConvertToJson.request(req);
|
||||
} catch (IllegalStateException e) {
|
||||
e.printStackTrace();
|
||||
unhandledKeys.add(key.toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
assertEquals(Collections.emptyList(), unhandledKeys, "Unhandled request keys");
|
||||
|
@ -101,14 +104,16 @@ public class RequestConvertToJsonTest {
|
|||
public void testAllResponseTypesHandled() {
|
||||
List<String> unhandledKeys = new ArrayList<>();
|
||||
for (ApiKeys key : ApiKeys.values()) {
|
||||
short version = key.latestVersion();
|
||||
ApiMessage message = ApiMessageType.fromApiKey(key.id).newResponse();
|
||||
ByteBuffer bytes = MessageUtil.toByteBuffer(message, version);
|
||||
AbstractResponse res = AbstractResponse.parseResponse(key, bytes, version);
|
||||
try {
|
||||
RequestConvertToJson.response(res, version);
|
||||
} catch (IllegalStateException e) {
|
||||
unhandledKeys.add(key.toString());
|
||||
if (key.hasValidVersion()) {
|
||||
short version = key.latestVersion();
|
||||
ApiMessage message = ApiMessageType.fromApiKey(key.id).newResponse();
|
||||
ByteBuffer bytes = MessageUtil.toByteBuffer(message, version);
|
||||
AbstractResponse res = AbstractResponse.parseResponse(key, bytes, version);
|
||||
try {
|
||||
RequestConvertToJson.response(res, version);
|
||||
} catch (IllegalStateException e) {
|
||||
unhandledKeys.add(key.toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
assertEquals(Collections.emptyList(), unhandledKeys, "Unhandled response keys");
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.kafka.common.errors.ClusterAuthorizationException;
|
|||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
|
||||
import org.apache.kafka.common.errors.TopicExistsException;
|
||||
import org.apache.kafka.common.internals.Topic;
|
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.requests.FetchRequest;
|
||||
import org.apache.kafka.common.requests.MetadataResponse;
|
||||
|
@ -57,6 +56,7 @@ import org.apache.kafka.common.test.api.ClusterTest;
|
|||
import org.apache.kafka.common.test.api.ClusterTestExtensions;
|
||||
import org.apache.kafka.common.test.api.Type;
|
||||
import org.apache.kafka.common.utils.Exit;
|
||||
import org.apache.kafka.metadata.LeaderAndIsr;
|
||||
import org.apache.kafka.server.common.AdminCommandFailedException;
|
||||
import org.apache.kafka.server.common.AdminOperationException;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
@ -1112,8 +1112,8 @@ public class TopicCommandTest {
|
|||
TestUtils.waitForCondition(
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(
|
||||
broker -> {
|
||||
Optional<UpdateMetadataRequestData.UpdateMetadataPartitionState> partitionState =
|
||||
Optional.ofNullable(broker.metadataCache().getPartitionInfo(testTopicName, 0).getOrElse(null));
|
||||
Optional<LeaderAndIsr> partitionState = Optional.ofNullable(
|
||||
broker.metadataCache().getLeaderAndIsr(testTopicName, 0).getOrElse(null));
|
||||
return partitionState.map(s -> FetchRequest.isValidBrokerId(s.leader())).orElse(false);
|
||||
}
|
||||
), CLUSTER_WAIT_MS, String.format("Meta data propogation fail in %s ms", CLUSTER_WAIT_MS));
|
||||
|
@ -1141,7 +1141,7 @@ public class TopicCommandTest {
|
|||
assertEquals(2, clusterInstance.aliveBrokers().size());
|
||||
|
||||
TestUtils.waitForCondition(
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(broker -> broker.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 2),
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(broker -> broker.metadataCache().getLeaderAndIsr(testTopicName, 0).get().isr().size() == 2),
|
||||
CLUSTER_WAIT_MS, String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName)
|
||||
);
|
||||
|
||||
|
@ -1244,7 +1244,7 @@ public class TopicCommandTest {
|
|||
assertEquals(4, clusterInstance.aliveBrokers().size());
|
||||
|
||||
TestUtils.waitForCondition(
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(broker -> broker.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 4),
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(broker -> broker.metadataCache().getLeaderAndIsr(testTopicName, 0).get().isr().size() == 4),
|
||||
CLUSTER_WAIT_MS, String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName)
|
||||
);
|
||||
|
||||
|
@ -1302,8 +1302,8 @@ public class TopicCommandTest {
|
|||
|
||||
TestUtils.waitForCondition(
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(broker ->
|
||||
broker.metadataCache().getPartitionInfo(underMinIsrTopic, 0).get().isr().size() < 6 &&
|
||||
broker.metadataCache().getPartitionInfo(offlineTopic, 0).get().leader() == MetadataResponse.NO_LEADER_ID),
|
||||
broker.metadataCache().getLeaderAndIsr(underMinIsrTopic, 0).get().isr().size() < 6 &&
|
||||
broker.metadataCache().getLeaderAndIsr(offlineTopic, 0).get().leader() == MetadataResponse.NO_LEADER_ID),
|
||||
CLUSTER_WAIT_MS, "Timeout waiting for partition metadata propagating to brokers for underMinIsrTopic topic"
|
||||
);
|
||||
|
||||
|
|
Loading…
Reference in New Issue