diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index 08b8d465a13db..a1973ad0edefd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -74,9 +74,8 @@ public ApiKeys apiKey() { return requestBuilder.apiKey(); } - public RequestHeader makeHeader() { - return new RequestHeader(requestBuilder.apiKey().id, - requestBuilder.version(), clientId, correlationId); + public RequestHeader makeHeader(short version) { + return new RequestHeader(requestBuilder.apiKey().id, version, clientId, correlationId); } public AbstractRequest.Builder requestBuilder() { diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 890bf56d1293e..4131bcb41d9a3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -280,43 +280,46 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long if (!canSendRequest(nodeId)) throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); } - AbstractRequest request = null; AbstractRequest.Builder builder = clientRequest.requestBuilder(); try { NodeApiVersions versionInfo = nodeApiVersions.get(nodeId); + short version; // Note: if versionInfo is null, we have no server version information. This would be // the case when sending the initial ApiVersionRequest which fetches the version // information itself. It is also the case when discoverBrokerVersions is set to false. if (versionInfo == null) { + version = builder.desiredOrLatestVersion(); if (discoverBrokerVersions && log.isTraceEnabled()) log.trace("No version information found when sending message of type {} to node {}. " + - "Assuming version {}.", clientRequest.apiKey(), nodeId, builder.version()); + "Assuming version {}.", clientRequest.apiKey(), nodeId, version); } else { - short version = versionInfo.usableVersion(clientRequest.apiKey()); - builder.setVersion(version); + version = versionInfo.usableVersion(clientRequest.apiKey()); } // The call to build may also throw UnsupportedVersionException, if there are essential // fields that cannot be represented in the chosen version. - request = builder.build(); + doSend(clientRequest, isInternalRequest, now, builder.build(version)); } catch (UnsupportedVersionException e) { // If the version is not supported, skip sending the request over the wire. // Instead, simply add it to the local queue of aborted requests. log.debug("Version mismatch when attempting to send {} to {}", clientRequest.toString(), clientRequest.destination(), e); - ClientResponse clientResponse = new ClientResponse(clientRequest.makeHeader(), + ClientResponse clientResponse = new ClientResponse(clientRequest.makeHeader(builder.desiredOrLatestVersion()), clientRequest.callback(), clientRequest.destination(), now, now, false, e, null); abortedSends.add(clientResponse); - return; } - RequestHeader header = clientRequest.makeHeader(); + } + + private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long now, AbstractRequest request) { + String nodeId = clientRequest.destination(); + RequestHeader header = clientRequest.makeHeader(request.version()); if (log.isDebugEnabled()) { int latestClientVersion = ProtoUtils.latestVersion(clientRequest.apiKey().id); if (header.apiVersion() == latestClientVersion) { log.trace("Sending {} to node {}.", request, nodeId); } else { log.debug("Using older server API v{} to send {} to node {}.", - header.apiVersion(), request, nodeId); + header.apiVersion(), request, nodeId); } } Send send = request.toSend(nodeId, header); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 0dc073e21f6e4..12ff9ce699113 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -697,11 +697,10 @@ private RequestFuture sendOffsetCommitRequest(final Map> sendListOffsetRequest(final Node node, - final Map timestampsToSearch, - boolean requireTimestamp) { - ListOffsetRequest.Builder builder = new ListOffsetRequest.Builder().setTargetTimes(timestampsToSearch); - - // If we need a timestamp in the response, the minimum RPC version we can send is v1. - // Otherwise, v0 is OK. - builder.setMinVersion(requireTimestamp ? (short) 1 : (short) 0); + final Map timestampsToSearch, + boolean requireTimestamp) { + // If we need a timestamp in the response, the minimum RPC version we can send is v1. Otherwise, v0 is OK. + short minVersion = requireTimestamp ? (short) 1 : (short) 0; + ListOffsetRequest.Builder builder = ListOffsetRequest.Builder.forConsumer(minVersion) + .setTargetTimes(timestampsToSearch); log.trace("Sending ListOffsetRequest {} to broker {}", builder, node); return client.send(node, builder) @@ -733,7 +732,7 @@ private Map createFetchRequests() { Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); - FetchRequest.Builder fetch = new FetchRequest.Builder(this.maxWaitMs, this.minBytes, entry.getValue()). + FetchRequest.Builder fetch = FetchRequest.Builder.forConsumer(this.maxWaitMs, this.minBytes, entry.getValue()). setMaxBytes(this.maxBytes); requests.put(node, fetch); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index f2ea4209355ec..eea1916cb7710 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -25,40 +25,39 @@ import java.nio.ByteBuffer; public abstract class AbstractRequest extends AbstractRequestResponse { - private final short version; public static abstract class Builder { private final ApiKeys apiKey; - private short version; + private final Short desiredVersion; public Builder(ApiKeys apiKey) { + this(apiKey, null); + } + + public Builder(ApiKeys apiKey, Short desiredVersion) { this.apiKey = apiKey; - this.version = ProtoUtils.latestVersion(apiKey.id); + this.desiredVersion = desiredVersion; } public ApiKeys apiKey() { return apiKey; } - public Builder setVersion(short version) { - this.version = version; - return this; + public short desiredOrLatestVersion() { + return desiredVersion == null ? ProtoUtils.latestVersion(apiKey.id) : desiredVersion; } - public short version() { - return version; + public T build() { + return build(desiredOrLatestVersion()); } - public abstract T build(); + public abstract T build(short version); } - public AbstractRequest(Struct struct, short version) { - super(struct); - this.version = version; - } + private final short version; - public Send toSend(String destination, RequestHeader header) { - return new NetworkSend(destination, serialize(header, this)); + public AbstractRequest(short version) { + this.version = version; } /** @@ -68,6 +67,19 @@ public short version() { return version; } + public Send toSend(String destination, RequestHeader header) { + return new NetworkSend(destination, serialize(header)); + } + + /** + * Use with care, typically {@link #toSend(String, RequestHeader)} should be used instead. + */ + public ByteBuffer serialize(RequestHeader header) { + return serialize(header.toStruct(), toStruct()); + } + + protected abstract Struct toStruct(); + /** * Get an error response for a request */ @@ -76,54 +88,78 @@ public short version() { /** * Factory method for getting a request object based on ApiKey ID and a buffer */ - public static AbstractRequest getRequest(int requestId, short versionId, ByteBuffer buffer) { + public static RequestAndSize getRequest(int requestId, short version, ByteBuffer buffer) { ApiKeys apiKey = ApiKeys.forId(requestId); + Struct struct = ProtoUtils.parseRequest(apiKey.id, version, buffer); + AbstractRequest request; switch (apiKey) { case PRODUCE: - return ProduceRequest.parse(buffer, versionId); + request = new ProduceRequest(struct, version); + break; case FETCH: - return FetchRequest.parse(buffer, versionId); + request = new FetchRequest(struct, version); + break; case LIST_OFFSETS: - return ListOffsetRequest.parse(buffer, versionId); + request = new ListOffsetRequest(struct, version); + break; case METADATA: - return MetadataRequest.parse(buffer, versionId); + request = new MetadataRequest(struct, version); + break; case OFFSET_COMMIT: - return OffsetCommitRequest.parse(buffer, versionId); + request = new OffsetCommitRequest(struct, version); + break; case OFFSET_FETCH: - return OffsetFetchRequest.parse(buffer, versionId); + request = new OffsetFetchRequest(struct, version); + break; case GROUP_COORDINATOR: - return GroupCoordinatorRequest.parse(buffer, versionId); + request = new GroupCoordinatorRequest(struct, version); + break; case JOIN_GROUP: - return JoinGroupRequest.parse(buffer, versionId); + request = new JoinGroupRequest(struct, version); + break; case HEARTBEAT: - return HeartbeatRequest.parse(buffer, versionId); + request = new HeartbeatRequest(struct, version); + break; case LEAVE_GROUP: - return LeaveGroupRequest.parse(buffer, versionId); + request = new LeaveGroupRequest(struct, version); + break; case SYNC_GROUP: - return SyncGroupRequest.parse(buffer, versionId); + request = new SyncGroupRequest(struct, version); + break; case STOP_REPLICA: - return StopReplicaRequest.parse(buffer, versionId); + request = new StopReplicaRequest(struct, version); + break; case CONTROLLED_SHUTDOWN_KEY: - return ControlledShutdownRequest.parse(buffer, versionId); + request = new ControlledShutdownRequest(struct, version); + break; case UPDATE_METADATA_KEY: - return UpdateMetadataRequest.parse(buffer, versionId); + request = new UpdateMetadataRequest(struct, version); + break; case LEADER_AND_ISR: - return LeaderAndIsrRequest.parse(buffer, versionId); + request = new LeaderAndIsrRequest(struct, version); + break; case DESCRIBE_GROUPS: - return DescribeGroupsRequest.parse(buffer, versionId); + request = new DescribeGroupsRequest(struct, version); + break; case LIST_GROUPS: - return ListGroupsRequest.parse(buffer, versionId); + request = new ListGroupsRequest(struct, version); + break; case SASL_HANDSHAKE: - return SaslHandshakeRequest.parse(buffer, versionId); + request = new SaslHandshakeRequest(struct, version); + break; case API_VERSIONS: - return ApiVersionsRequest.parse(buffer, versionId); + request = new ApiVersionsRequest(struct, version); + break; case CREATE_TOPICS: - return CreateTopicsRequest.parse(buffer, versionId); + request = new CreateTopicsRequest(struct, version); + break; case DELETE_TOPICS: - return DeleteTopicsRequest.parse(buffer, versionId); + request = new DeleteTopicsRequest(struct, version); + break; default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `getRequest`, the " + "code should be updated to do so.", apiKey)); } + return new RequestAndSize(request, struct.sizeOf()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java index 3ad16a54575c4..00ddf71dc5737 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java @@ -17,56 +17,13 @@ import java.nio.ByteBuffer; public abstract class AbstractRequestResponse { - protected final Struct struct; - - public AbstractRequestResponse(Struct struct) { - this.struct = struct; - } - - public Struct toStruct() { - return struct; - } - - /** - * Get the serialized size of this object - */ - public int sizeOf() { - return struct.sizeOf(); - } - /** - * Write this object to a buffer + * Visible for testing. */ - public void writeTo(ByteBuffer buffer) { - struct.writeTo(buffer); - } - - @Override - public String toString() { - return struct.toString(); - } - - @Override - public int hashCode() { - return struct.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - AbstractRequestResponse other = (AbstractRequestResponse) obj; - return struct.equals(other.struct); - } - - public static ByteBuffer serialize(AbstractRequestResponse header, AbstractRequestResponse body) { - ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf()); - header.writeTo(buffer); - body.writeTo(buffer); + public static ByteBuffer serialize(Struct headerStruct, Struct bodyStruct) { + ByteBuffer buffer = ByteBuffer.allocate(headerStruct.sizeOf() + bodyStruct.sizeOf()); + headerStruct.writeTo(buffer); + bodyStruct.writeTo(buffer); buffer.rewind(); return buffer; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 8bbc25a40e3c5..a21e340080c7d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -21,17 +21,32 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.types.Struct; +import java.nio.ByteBuffer; + public abstract class AbstractResponse extends AbstractRequestResponse { - public AbstractResponse(Struct struct) { - super(struct); + public Send toSend(String destination, RequestHeader requestHeader) { + return toSend(destination, requestHeader.apiVersion(), requestHeader.toResponseHeader()); + } + + /** + * This should only be used if we need to return a response with a different version than the request, which + * should be very rare (an example is @link {@link ApiVersionsResponse#unsupportedVersionSend(String, RequestHeader)}). + * Typically {@link #toSend(String, RequestHeader)} should be used. + */ + public Send toSend(String destination, short version, ResponseHeader responseHeader) { + return new NetworkSend(destination, serialize(version, responseHeader)); } - public Send toSend(String destination, RequestHeader request) { - ResponseHeader responseHeader = new ResponseHeader(request.correlationId()); - return new NetworkSend(destination, serialize(responseHeader, this)); + /** + * Visible for testing, typically {@link #toSend(String, RequestHeader)} should be used instead. + */ + public ByteBuffer serialize(short version, ResponseHeader responseHeader) { + return serialize(responseHeader.toStruct(), toStruct(version)); } + protected abstract Struct toStruct(short version); + public static AbstractResponse getResponse(int requestId, Struct struct) { ApiKeys apiKey = ApiKeys.forId(requestId); switch (apiKey) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java index fe7c3488a3dc7..7d409007a9cf3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java @@ -22,13 +22,14 @@ public class ApiVersionsRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { + public Builder() { super(ApiKeys.API_VERSIONS); } @Override - public ApiVersionsRequest build() { - return new ApiVersionsRequest(version()); + public ApiVersionsRequest build(short version) { + return new ApiVersionsRequest(version); } @Override @@ -38,12 +39,16 @@ public String toString() { } public ApiVersionsRequest(short version) { - this(new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, version)), - version); + super(version); } - public ApiVersionsRequest(Struct struct, short versionId) { - super(struct, versionId); + public ApiVersionsRequest(Struct struct, short version) { + super(version); + } + + @Override + protected Struct toStruct() { + return new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, version())); } @Override @@ -58,13 +63,8 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static ApiVersionsRequest parse(ByteBuffer buffer, int versionId) { - return new ApiVersionsRequest( - ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, versionId, buffer), - (short) versionId); + public static ApiVersionsRequest parse(ByteBuffer buffer, short version) { + return new ApiVersionsRequest(ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, version, buffer), version); } - public static ApiVersionsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.API_VERSIONS.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 7d8bcc54efbc8..0066c084467e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -12,10 +12,10 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -28,8 +28,6 @@ public class ApiVersionsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.API_VERSIONS.id); - public static final ApiVersionsResponse API_VERSIONS_RESPONSE = createApiVersionsResponse(); public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String API_VERSIONS_KEY_NAME = "api_versions"; @@ -67,23 +65,11 @@ public String toString() { } public ApiVersionsResponse(Errors error, List apiVersions) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - List apiVersionList = new ArrayList<>(); - for (ApiVersion apiVersion : apiVersions) { - Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME); - apiVersionStruct.set(API_KEY_NAME, apiVersion.apiKey); - apiVersionStruct.set(MIN_VERSION_KEY_NAME, apiVersion.minVersion); - apiVersionStruct.set(MAX_VERSION_KEY_NAME, apiVersion.maxVersion); - apiVersionList.add(apiVersionStruct); - } - struct.set(API_VERSIONS_KEY_NAME, apiVersionList.toArray()); this.error = error; this.apiKeyToApiVersion = buildApiKeyToApiVersion(apiVersions); } public ApiVersionsResponse(Struct struct) { - super(struct); this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); List tempApiVersions = new ArrayList<>(); for (Object apiVersionsObj : struct.getArray(API_VERSIONS_KEY_NAME)) { @@ -96,6 +82,31 @@ public ApiVersionsResponse(Struct struct) { this.apiKeyToApiVersion = buildApiKeyToApiVersion(tempApiVersions); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.API_VERSIONS.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + List apiVersionList = new ArrayList<>(); + for (ApiVersion apiVersion : apiKeyToApiVersion.values()) { + Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME); + apiVersionStruct.set(API_KEY_NAME, apiVersion.apiKey); + apiVersionStruct.set(MIN_VERSION_KEY_NAME, apiVersion.minVersion); + apiVersionStruct.set(MAX_VERSION_KEY_NAME, apiVersion.maxVersion); + apiVersionList.add(apiVersionStruct); + } + struct.set(API_VERSIONS_KEY_NAME, apiVersionList.toArray()); + return struct; + } + + /** + * Returns Errors.UNSUPPORTED_VERSION response with version 0 since we don't support the requested version. + */ + public static Send unsupportedVersionSend(String destination, RequestHeader requestHeader) { + ApiVersionsResponse response = new ApiVersionsResponse(Errors.UNSUPPORTED_VERSION, + Collections.emptyList()); + return response.toSend(destination, (short) 0, requestHeader.toResponseHeader()); + } + public Collection apiVersions() { return apiKeyToApiVersion.values(); } @@ -108,12 +119,8 @@ public Errors error() { return error; } - public static ApiVersionsResponse parse(ByteBuffer buffer) { - return new ApiVersionsResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static ApiVersionsResponse fromError(Errors error) { - return new ApiVersionsResponse(error, Collections.emptyList()); + public static ApiVersionsResponse parse(ByteBuffer buffer, short version) { + return new ApiVersionsResponse(ProtoUtils.responseSchema(ApiKeys.API_VERSIONS.id, version).read(buffer)); } private static ApiVersionsResponse createApiVersionsResponse() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java index 8f44e5cfe7c92..679e5dd327958 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java @@ -33,8 +33,8 @@ public Builder(int brokerId) { } @Override - public ControlledShutdownRequest build() { - return new ControlledShutdownRequest(brokerId, version()); + public ControlledShutdownRequest build(short version) { + return new ControlledShutdownRequest(brokerId, version); } @Override @@ -49,14 +49,12 @@ public String toString() { private int brokerId; private ControlledShutdownRequest(int brokerId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version)), - version); - struct.set(BROKER_ID_KEY_NAME, brokerId); + super(version); this.brokerId = brokerId; } - public ControlledShutdownRequest(Struct struct, short versionId) { - super(struct, versionId); + public ControlledShutdownRequest(Struct struct, short version) { + super(version); brokerId = struct.getInt(BROKER_ID_KEY_NAME); } @@ -79,12 +77,15 @@ public int brokerId() { return brokerId; } - public static ControlledShutdownRequest parse(ByteBuffer buffer, int versionId) { + public static ControlledShutdownRequest parse(ByteBuffer buffer, short versionId) { return new ControlledShutdownRequest( - ProtoUtils.parseRequest(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, versionId, buffer), (short) versionId); + ProtoUtils.parseRequest(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, versionId, buffer), versionId); } - public static ControlledShutdownRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version())); + struct.set(BROKER_ID_KEY_NAME, brokerId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java index b3922f9573542..a2cac6c9f096f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,8 +26,6 @@ public class ControlledShutdownResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); - private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String PARTITIONS_REMAINING_KEY_NAME = "partitions_remaining"; @@ -47,25 +44,11 @@ public class ControlledShutdownResponse extends AbstractResponse { private final Set partitionsRemaining; public ControlledShutdownResponse(Errors error, Set partitionsRemaining) { - super(new Struct(CURRENT_SCHEMA)); - - struct.set(ERROR_CODE_KEY_NAME, error.code()); - - List partitionsRemainingList = new ArrayList<>(partitionsRemaining.size()); - for (TopicPartition topicPartition : partitionsRemaining) { - Struct topicPartitionStruct = struct.instance(PARTITIONS_REMAINING_KEY_NAME); - topicPartitionStruct.set(TOPIC_KEY_NAME, topicPartition.topic()); - topicPartitionStruct.set(PARTITION_KEY_NAME, topicPartition.partition()); - partitionsRemainingList.add(topicPartitionStruct); - } - struct.set(PARTITIONS_REMAINING_KEY_NAME, partitionsRemainingList.toArray()); - this.error = error; this.partitionsRemaining = partitionsRemaining; } public ControlledShutdownResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); Set partitions = new HashSet<>(); for (Object topicPartitionObj : struct.getArray(PARTITIONS_REMAINING_KEY_NAME)) { @@ -85,12 +68,25 @@ public Set partitionsRemaining() { return partitionsRemaining; } - public static ControlledShutdownResponse parse(ByteBuffer buffer) { - return new ControlledShutdownResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static ControlledShutdownResponse parse(ByteBuffer buffer, int version) { + public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) { return new ControlledShutdownResponse(ProtoUtils.parseResponse(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version, buffer)); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version)); + + struct.set(ERROR_CODE_KEY_NAME, error.code()); + + List partitionsRemainingList = new ArrayList<>(partitionsRemaining.size()); + for (TopicPartition topicPartition : partitionsRemaining) { + Struct topicPartitionStruct = struct.instance(PARTITIONS_REMAINING_KEY_NAME); + topicPartitionStruct.set(TOPIC_KEY_NAME, topicPartition.topic()); + topicPartitionStruct.set(PARTITION_KEY_NAME, topicPartition.partition()); + partitionsRemainingList.add(topicPartitionStruct); + } + struct.set(PARTITIONS_REMAINING_KEY_NAME, partitionsRemainingList.toArray()); + + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index 0a4bce01394ac..a8f8c5e9ece31 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -113,11 +113,11 @@ public Builder(Map topics, int timeout, boolean validateOn } @Override - public CreateTopicsRequest build() { - if (validateOnly && version() == 0) + public CreateTopicsRequest build(short version) { + if (validateOnly && version == 0) throw new UnsupportedVersionException("validateOnly is not supported in version 0 of " + "CreateTopicsRequest"); - return new CreateTopicsRequest(topics, timeout, validateOnly, version()); + return new CreateTopicsRequest(topics, timeout, validateOnly, version); } @Override @@ -144,53 +144,15 @@ public String toString() { public static final short NO_REPLICATION_FACTOR = -1; private CreateTopicsRequest(Map topics, Integer timeout, boolean validateOnly, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.CREATE_TOPICS.id, version)), version); - - List createTopicRequestStructs = new ArrayList<>(topics.size()); - for (Map.Entry entry : topics.entrySet()) { - - Struct singleRequestStruct = struct.instance(REQUESTS_KEY_NAME); - String topic = entry.getKey(); - TopicDetails args = entry.getValue(); - - singleRequestStruct.set(TOPIC_KEY_NAME, topic); - singleRequestStruct.set(NUM_PARTITIONS_KEY_NAME, args.numPartitions); - singleRequestStruct.set(REPLICATION_FACTOR_KEY_NAME, args.replicationFactor); - - // replica assignment - List replicaAssignmentsStructs = new ArrayList<>(args.replicasAssignments.size()); - for (Map.Entry> partitionReplicaAssignment : args.replicasAssignments.entrySet()) { - Struct replicaAssignmentStruct = singleRequestStruct.instance(REPLICA_ASSIGNMENT_KEY_NAME); - replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME, partitionReplicaAssignment.getKey()); - replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME, partitionReplicaAssignment.getValue().toArray()); - replicaAssignmentsStructs.add(replicaAssignmentStruct); - } - singleRequestStruct.set(REPLICA_ASSIGNMENT_KEY_NAME, replicaAssignmentsStructs.toArray()); - - // configs - List configsStructs = new ArrayList<>(args.configs.size()); - for (Map.Entry configEntry : args.configs.entrySet()) { - Struct configStruct = singleRequestStruct.instance(CONFIGS_KEY_NAME); - configStruct.set(CONFIG_KEY_KEY_NAME, configEntry.getKey()); - configStruct.set(CONFIG_VALUE_KEY_NAME, configEntry.getValue()); - configsStructs.add(configStruct); - } - singleRequestStruct.set(CONFIGS_KEY_NAME, configsStructs.toArray()); - createTopicRequestStructs.add(singleRequestStruct); - } - struct.set(REQUESTS_KEY_NAME, createTopicRequestStructs.toArray()); - struct.set(TIMEOUT_KEY_NAME, timeout); - if (version >= 1) - struct.set(VALIDATE_ONLY_KEY_NAME, validateOnly); - + super(version); this.topics = topics; this.timeout = timeout; this.validateOnly = validateOnly; this.duplicateTopics = Collections.emptySet(); } - public CreateTopicsRequest(Struct struct, short versionId) { - super(struct, versionId); + public CreateTopicsRequest(Struct struct, short version) { + super(version); Object[] requestStructs = struct.getArray(REQUESTS_KEY_NAME); Map topics = new HashMap<>(); @@ -262,7 +224,7 @@ public AbstractResponse getErrorResponse(Throwable e) { switch (versionId) { case 0: case 1: - return new CreateTopicsResponse(topicErrors, versionId); + return new CreateTopicsResponse(topicErrors); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CREATE_TOPICS.id))); @@ -285,13 +247,55 @@ public Set duplicateTopics() { return this.duplicateTopics; } - public static CreateTopicsRequest parse(ByteBuffer buffer, int versionId) { - return new CreateTopicsRequest( - ProtoUtils.parseRequest(ApiKeys.CREATE_TOPICS.id, versionId, buffer), - (short) versionId); + public static CreateTopicsRequest parse(ByteBuffer buffer, short versionId) { + return new CreateTopicsRequest(ProtoUtils.parseRequest(ApiKeys.CREATE_TOPICS.id, versionId, buffer), versionId); } - public static CreateTopicsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.CREATE_TOPICS.id)); + /** + * Visible for testing. + */ + @Override + public Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.CREATE_TOPICS.id, version)); + + List createTopicRequestStructs = new ArrayList<>(topics.size()); + for (Map.Entry entry : topics.entrySet()) { + + Struct singleRequestStruct = struct.instance(REQUESTS_KEY_NAME); + String topic = entry.getKey(); + TopicDetails args = entry.getValue(); + + singleRequestStruct.set(TOPIC_KEY_NAME, topic); + singleRequestStruct.set(NUM_PARTITIONS_KEY_NAME, args.numPartitions); + singleRequestStruct.set(REPLICATION_FACTOR_KEY_NAME, args.replicationFactor); + + // replica assignment + List replicaAssignmentsStructs = new ArrayList<>(args.replicasAssignments.size()); + for (Map.Entry> partitionReplicaAssignment : args.replicasAssignments.entrySet()) { + Struct replicaAssignmentStruct = singleRequestStruct.instance(REPLICA_ASSIGNMENT_KEY_NAME); + replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME, partitionReplicaAssignment.getKey()); + replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME, partitionReplicaAssignment.getValue().toArray()); + replicaAssignmentsStructs.add(replicaAssignmentStruct); + } + singleRequestStruct.set(REPLICA_ASSIGNMENT_KEY_NAME, replicaAssignmentsStructs.toArray()); + + // configs + List configsStructs = new ArrayList<>(args.configs.size()); + for (Map.Entry configEntry : args.configs.entrySet()) { + Struct configStruct = singleRequestStruct.instance(CONFIGS_KEY_NAME); + configStruct.set(CONFIG_KEY_KEY_NAME, configEntry.getKey()); + configStruct.set(CONFIG_VALUE_KEY_NAME, configEntry.getValue()); + configsStructs.add(configStruct); + } + singleRequestStruct.set(CONFIGS_KEY_NAME, configsStructs.toArray()); + createTopicRequestStructs.add(singleRequestStruct); + } + struct.set(REQUESTS_KEY_NAME, createTopicRequestStructs.toArray()); + struct.set(TIMEOUT_KEY_NAME, timeout); + if (version >= 1) + struct.set(VALIDATE_ONLY_KEY_NAME, validateOnly); + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index b09795f3d32cc..01b7c2b17d577 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -30,8 +29,6 @@ import java.util.Map; public class CreateTopicsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CREATE_TOPICS.id); - private static final String TOPIC_ERRORS_KEY_NAME = "topic_errors"; private static final String TOPIC_KEY_NAME = "topic"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -87,27 +84,11 @@ public String toString() { private final Map errors; - public CreateTopicsResponse(Map errors, short version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version))); - - List topicErrorsStructs = new ArrayList<>(errors.size()); - for (Map.Entry topicError : errors.entrySet()) { - Struct topicErrorsStruct = struct.instance(TOPIC_ERRORS_KEY_NAME); - topicErrorsStruct.set(TOPIC_KEY_NAME, topicError.getKey()); - Error error = topicError.getValue(); - topicErrorsStruct.set(ERROR_CODE_KEY_NAME, error.error.code()); - if (version >= 1) - topicErrorsStruct.set(ERROR_MESSAGE_KEY_NAME, error.message()); - topicErrorsStructs.add(topicErrorsStruct); - } - struct.set(TOPIC_ERRORS_KEY_NAME, topicErrorsStructs.toArray()); - + public CreateTopicsResponse(Map errors) { this.errors = errors; } public CreateTopicsResponse(Struct struct) { - super(struct); - Object[] topicErrorStructs = struct.getArray(TOPIC_ERRORS_KEY_NAME); Map errors = new HashMap<>(); for (Object topicErrorStructObj : topicErrorStructs) { @@ -123,15 +104,29 @@ public CreateTopicsResponse(Struct struct) { this.errors = errors; } - public Map errors() { - return errors; + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version)); + + List topicErrorsStructs = new ArrayList<>(errors.size()); + for (Map.Entry topicError : errors.entrySet()) { + Struct topicErrorsStruct = struct.instance(TOPIC_ERRORS_KEY_NAME); + topicErrorsStruct.set(TOPIC_KEY_NAME, topicError.getKey()); + Error error = topicError.getValue(); + topicErrorsStruct.set(ERROR_CODE_KEY_NAME, error.error.code()); + if (version >= 1) + topicErrorsStruct.set(ERROR_MESSAGE_KEY_NAME, error.message()); + topicErrorsStructs.add(topicErrorsStruct); + } + struct.set(TOPIC_ERRORS_KEY_NAME, topicErrorsStructs.toArray()); + return struct; } - public static CreateTopicsResponse parse(ByteBuffer buffer) { - return new CreateTopicsResponse(CURRENT_SCHEMA.read(buffer)); + public Map errors() { + return errors; } - public static CreateTopicsResponse parse(ByteBuffer buffer, int version) { + public static CreateTopicsResponse parse(ByteBuffer buffer, short version) { return new CreateTopicsResponse(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java index 2874fadc7a99b..eea4aa93cd60d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java @@ -46,8 +46,8 @@ public Builder(Set topics, Integer timeout) { } @Override - public DeleteTopicsRequest build() { - return new DeleteTopicsRequest(topics, timeout, version()); + public DeleteTopicsRequest build(short version) { + return new DeleteTopicsRequest(topics, timeout, version); } @Override @@ -62,17 +62,13 @@ public String toString() { } private DeleteTopicsRequest(Set topics, Integer timeout, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.DELETE_TOPICS.id, version)), version); - - struct.set(TOPICS_KEY_NAME, topics.toArray()); - struct.set(TIMEOUT_KEY_NAME, timeout); - + super(version); this.topics = topics; this.timeout = timeout; } public DeleteTopicsRequest(Struct struct, short version) { - super(struct, version); + super(version); Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME); Set topics = new HashSet<>(topicsArray.length); for (Object topic : topicsArray) @@ -82,6 +78,14 @@ public DeleteTopicsRequest(Struct struct, short version) { this.timeout = struct.getInt(TIMEOUT_KEY_NAME); } + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.DELETE_TOPICS.id, version())); + struct.set(TOPICS_KEY_NAME, topics.toArray()); + struct.set(TIMEOUT_KEY_NAME, timeout); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { Map topicErrors = new HashMap<>(); @@ -105,12 +109,8 @@ public Integer timeout() { return this.timeout; } - public static DeleteTopicsRequest parse(ByteBuffer buffer, int versionId) { - return new DeleteTopicsRequest(ProtoUtils.parseRequest(ApiKeys.DELETE_TOPICS.id, versionId, buffer), - (short) versionId); + public static DeleteTopicsRequest parse(ByteBuffer buffer, short versionId) { + return new DeleteTopicsRequest(ProtoUtils.parseRequest(ApiKeys.DELETE_TOPICS.id, versionId, buffer), versionId); } - public static DeleteTopicsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java index 5c8b3d5a5dc53..c47d0982b248c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -29,7 +28,6 @@ import java.util.Map; public class DeleteTopicsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.DELETE_TOPICS.id); private static final String TOPIC_ERROR_CODES_KEY_NAME = "topic_error_codes"; private static final String TOPIC_KEY_NAME = "topic"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -45,23 +43,10 @@ public class DeleteTopicsResponse extends AbstractResponse { private final Map errors; public DeleteTopicsResponse(Map errors) { - super(new Struct(CURRENT_SCHEMA)); - - List topicErrorCodeStructs = new ArrayList<>(errors.size()); - for (Map.Entry topicError : errors.entrySet()) { - Struct topicErrorCodeStruct = struct.instance(TOPIC_ERROR_CODES_KEY_NAME); - topicErrorCodeStruct.set(TOPIC_KEY_NAME, topicError.getKey()); - topicErrorCodeStruct.set(ERROR_CODE_KEY_NAME, topicError.getValue().code()); - topicErrorCodeStructs.add(topicErrorCodeStruct); - } - struct.set(TOPIC_ERROR_CODES_KEY_NAME, topicErrorCodeStructs.toArray()); - this.errors = errors; } public DeleteTopicsResponse(Struct struct) { - super(struct); - Object[] topicErrorCodesStructs = struct.getArray(TOPIC_ERROR_CODES_KEY_NAME); Map errors = new HashMap<>(); for (Object topicErrorCodeStructObj : topicErrorCodesStructs) { @@ -74,15 +59,25 @@ public DeleteTopicsResponse(Struct struct) { this.errors = errors; } - public Map errors() { - return errors; + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.DELETE_TOPICS.id, version)); + List topicErrorCodeStructs = new ArrayList<>(errors.size()); + for (Map.Entry topicError : errors.entrySet()) { + Struct topicErrorCodeStruct = struct.instance(TOPIC_ERROR_CODES_KEY_NAME); + topicErrorCodeStruct.set(TOPIC_KEY_NAME, topicError.getKey()); + topicErrorCodeStruct.set(ERROR_CODE_KEY_NAME, topicError.getValue().code()); + topicErrorCodeStructs.add(topicErrorCodeStruct); + } + struct.set(TOPIC_ERROR_CODES_KEY_NAME, topicErrorCodeStructs.toArray()); + return struct; } - public static DeleteTopicsResponse parse(ByteBuffer buffer) { - return new DeleteTopicsResponse(CURRENT_SCHEMA.read(buffer)); + public Map errors() { + return errors; } - public static DeleteTopicsResponse parse(ByteBuffer buffer, int version) { + public static DeleteTopicsResponse parse(ByteBuffer buffer, short version) { return new DeleteTopicsResponse(ProtoUtils.responseSchema(ApiKeys.DELETE_TOPICS.id, version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java index f17cdd90b3021..0f1337101dc81 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java @@ -34,8 +34,7 @@ public Builder(List groupIds) { } @Override - public DescribeGroupsRequest build() { - short version = version(); + public DescribeGroupsRequest build(short version) { return new DescribeGroupsRequest(this.groupIds, version); } @@ -48,14 +47,12 @@ public String toString() { private final List groupIds; private DescribeGroupsRequest(List groupIds, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.DESCRIBE_GROUPS.id, version)), - version); - struct.set(GROUP_IDS_KEY_NAME, groupIds.toArray()); + super(version); this.groupIds = groupIds; } public DescribeGroupsRequest(Struct struct, short version) { - super(struct, version); + super(version); this.groupIds = new ArrayList<>(); for (Object groupId : struct.getArray(GROUP_IDS_KEY_NAME)) this.groupIds.add((String) groupId); @@ -65,6 +62,13 @@ public List groupIds() { return groupIds; } + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.DESCRIBE_GROUPS.id, version())); + struct.set(GROUP_IDS_KEY_NAME, groupIds.toArray()); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { short versionId = version(); @@ -78,12 +82,7 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static DescribeGroupsRequest parse(ByteBuffer buffer, int versionId) { - return new DescribeGroupsRequest(ProtoUtils.parseRequest(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer), - (short) versionId); - } - - public static DescribeGroupsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.DESCRIBE_GROUPS.id)); + public static DescribeGroupsRequest parse(ByteBuffer buffer, short versionId) { + return new DescribeGroupsRequest(ProtoUtils.parseRequest(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer), versionId); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index 56b387e37496c..0dde987476de8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,8 +26,6 @@ public class DescribeGroupsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.DESCRIBE_GROUPS.id); - private static final String GROUPS_KEY_NAME = "groups"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -60,36 +57,10 @@ public class DescribeGroupsResponse extends AbstractResponse { private final Map groups; public DescribeGroupsResponse(Map groups) { - super(new Struct(CURRENT_SCHEMA)); - - List groupStructs = new ArrayList<>(); - for (Map.Entry groupEntry : groups.entrySet()) { - Struct groupStruct = struct.instance(GROUPS_KEY_NAME); - GroupMetadata group = groupEntry.getValue(); - groupStruct.set(GROUP_ID_KEY_NAME, groupEntry.getKey()); - groupStruct.set(ERROR_CODE_KEY_NAME, group.error.code()); - groupStruct.set(GROUP_STATE_KEY_NAME, group.state); - groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); - groupStruct.set(PROTOCOL_KEY_NAME, group.protocol); - List membersList = new ArrayList<>(); - for (GroupMember member : group.members) { - Struct memberStruct = groupStruct.instance(MEMBERS_KEY_NAME); - memberStruct.set(MEMBER_ID_KEY_NAME, member.memberId); - memberStruct.set(CLIENT_ID_KEY_NAME, member.clientId); - memberStruct.set(CLIENT_HOST_KEY_NAME, member.clientHost); - memberStruct.set(MEMBER_METADATA_KEY_NAME, member.memberMetadata); - memberStruct.set(MEMBER_ASSIGNMENT_KEY_NAME, member.memberAssignment); - membersList.add(memberStruct); - } - groupStruct.set(MEMBERS_KEY_NAME, membersList.toArray()); - groupStructs.add(groupStruct); - } - struct.set(GROUPS_KEY_NAME, groupStructs.toArray()); this.groups = groups; } public DescribeGroupsResponse(Struct struct) { - super(struct); this.groups = new HashMap<>(); for (Object groupObj : struct.getArray(GROUPS_KEY_NAME)) { Struct groupStruct = (Struct) groupObj; @@ -209,10 +180,6 @@ public ByteBuffer memberAssignment() { } } - public static DescribeGroupsResponse parse(ByteBuffer buffer) { - return new DescribeGroupsResponse(CURRENT_SCHEMA.read(buffer)); - } - public static DescribeGroupsResponse fromError(Errors error, List groupIds) { GroupMetadata errorMetadata = GroupMetadata.forError(error); Map groups = new HashMap<>(); @@ -221,4 +188,38 @@ public static DescribeGroupsResponse fromError(Errors error, List groupI return new DescribeGroupsResponse(groups); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.DESCRIBE_GROUPS.id, version)); + + List groupStructs = new ArrayList<>(); + for (Map.Entry groupEntry : groups.entrySet()) { + Struct groupStruct = struct.instance(GROUPS_KEY_NAME); + GroupMetadata group = groupEntry.getValue(); + groupStruct.set(GROUP_ID_KEY_NAME, groupEntry.getKey()); + groupStruct.set(ERROR_CODE_KEY_NAME, group.error.code()); + groupStruct.set(GROUP_STATE_KEY_NAME, group.state); + groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); + groupStruct.set(PROTOCOL_KEY_NAME, group.protocol); + List membersList = new ArrayList<>(); + for (GroupMember member : group.members) { + Struct memberStruct = groupStruct.instance(MEMBERS_KEY_NAME); + memberStruct.set(MEMBER_ID_KEY_NAME, member.memberId); + memberStruct.set(CLIENT_ID_KEY_NAME, member.clientId); + memberStruct.set(CLIENT_HOST_KEY_NAME, member.clientHost); + memberStruct.set(MEMBER_METADATA_KEY_NAME, member.memberMetadata); + memberStruct.set(MEMBER_ASSIGNMENT_KEY_NAME, member.memberAssignment); + membersList.add(memberStruct); + } + groupStruct.set(MEMBERS_KEY_NAME, membersList.toArray()); + groupStructs.add(groupStruct); + } + struct.set(GROUPS_KEY_NAME, groupStructs.toArray()); + + return struct; + } + + public static DescribeGroupsResponse parse(ByteBuffer buffer, short versionId) { + return new DescribeGroupsResponse(ProtoUtils.parseResponse(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer)); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 4f270e15043cf..55fd286d631f8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -92,27 +92,32 @@ public static List> batchByTopic(LinkedHashMap { - private int replicaId = CONSUMER_REPLICA_ID; - private int maxWait; + private final int maxWait; private final int minBytes; + private final int replicaId; + private final LinkedHashMap fetchData; private int maxBytes = DEFAULT_RESPONSE_MAX_BYTES; - private LinkedHashMap fetchData; - public Builder(int maxWait, int minBytes, LinkedHashMap fetchData) { - super(ApiKeys.FETCH); - this.maxWait = maxWait; - this.minBytes = minBytes; - this.fetchData = fetchData; + public static Builder forConsumer(int maxWait, int minBytes, LinkedHashMap fetchData) { + return new Builder(null, CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData); } - public Builder setReplicaId(int replicaId) { - this.replicaId = replicaId; - return this; + public static Builder forReplica(short desiredVersion, int replicaId, int maxWait, int minBytes, + LinkedHashMap fetchData) { + return new Builder(desiredVersion, replicaId, maxWait, minBytes, fetchData); } - public Builder setMaxWait(int maxWait) { + private Builder(Short desiredVersion, int replicaId, int maxWait, int minBytes, + LinkedHashMap fetchData) { + super(ApiKeys.FETCH, desiredVersion); + this.replicaId = replicaId; this.maxWait = maxWait; - return this; + this.minBytes = minBytes; + this.fetchData = fetchData; + } + + public LinkedHashMap fetchData() { + return this.fetchData; } public Builder setMaxBytes(int maxBytes) { @@ -120,19 +125,13 @@ public Builder setMaxBytes(int maxBytes) { return this; } - public LinkedHashMap fetchData() { - return this.fetchData; - } - @Override - public FetchRequest build() { - short version = version(); + public FetchRequest build(short version) { if (version < 3) { maxBytes = -1; } - return new FetchRequest(version, replicaId, maxWait, minBytes, - maxBytes, fetchData); + return new FetchRequest(version, replicaId, maxWait, minBytes, maxBytes, fetchData); } @Override @@ -151,31 +150,7 @@ public String toString() { private FetchRequest(short version, int replicaId, int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version)), version); - List> topicsData = TopicAndPartitionData.batchByTopic(fetchData); - - struct.set(REPLICA_ID_KEY_NAME, replicaId); - struct.set(MAX_WAIT_KEY_NAME, maxWait); - struct.set(MIN_BYTES_KEY_NAME, minBytes); - if (version >= 3) - struct.set(MAX_BYTES_KEY_NAME, maxBytes); - List topicArray = new ArrayList<>(); - for (TopicAndPartitionData topicEntry : topicsData) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.topic); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.partitions.entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + super(version); this.replicaId = replicaId; this.maxWait = maxWait; this.minBytes = minBytes; @@ -183,8 +158,8 @@ private FetchRequest(short version, int replicaId, int maxWait, int minBytes, in this.fetchData = fetchData; } - public FetchRequest(Struct struct, short versionId) { - super(struct, versionId); + public FetchRequest(Struct struct, short version) { + super(version); replicaId = struct.getInt(REPLICA_ID_KEY_NAME); maxWait = struct.getInt(MAX_WAIT_KEY_NAME); minBytes = struct.getInt(MIN_BYTES_KEY_NAME); @@ -217,8 +192,7 @@ public AbstractResponse getErrorResponse(Throwable e) { responseData.put(entry.getKey(), partitionResponse); } - short versionId = version(); - return new FetchResponse(versionId, responseData, 0); + return new FetchResponse(responseData, 0); } public int replicaId() { @@ -245,11 +219,38 @@ public boolean isFromFollower() { return replicaId >= 0; } - public static FetchRequest parse(ByteBuffer buffer, int versionId) { - return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer), (short) versionId); + public static FetchRequest parse(ByteBuffer buffer, short versionId) { + return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer), versionId); } - public static FetchRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.FETCH.id)); + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version)); + List> topicsData = TopicAndPartitionData.batchByTopic(fetchData); + + struct.set(REPLICA_ID_KEY_NAME, replicaId); + struct.set(MAX_WAIT_KEY_NAME, maxWait); + struct.set(MIN_BYTES_KEY_NAME, minBytes); + if (version >= 3) + struct.set(MAX_BYTES_KEY_NAME, maxBytes); + List topicArray = new ArrayList<>(); + for (TopicAndPartitionData topicEntry : topicsData) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.topic); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.partitions.entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 64bd3d3aa360b..dee6a5f53b496 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -23,7 +23,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; + import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; import org.apache.kafka.common.record.Records; @@ -39,7 +39,6 @@ */ public class FetchResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names @@ -71,7 +70,7 @@ public class FetchResponse extends AbstractResponse { public static final long INVALID_HIGHWATERMARK = -1L; private final LinkedHashMap responseData; - private final int throttleTime; + private final int throttleTimeMs; public static final class PartitionData { public final Errors error; @@ -91,18 +90,6 @@ public String toString() { } } - /** - * Constructor for version 3. - * - * The entries in `responseData` should be in the same order as the entries in `FetchRequest.fetchData`. - * - * @param responseData fetched data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled - */ - public FetchResponse(LinkedHashMap responseData, int throttleTime) { - this(3, responseData, throttleTime); - } - /** * Constructor for all versions. * @@ -110,17 +97,14 @@ public FetchResponse(LinkedHashMap responseData, * `FetchRequest.fetchData`. * * @param responseData fetched data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled + * @param throttleTimeMs Time in milliseconds the response was throttled */ - public FetchResponse(int version, LinkedHashMap responseData, int throttleTime) { - super(writeStruct(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)), version, responseData, - throttleTime)); + public FetchResponse(LinkedHashMap responseData, int throttleTimeMs) { this.responseData = responseData; - this.throttleTime = throttleTime; + this.throttleTimeMs = throttleTimeMs; } public FetchResponse(Struct struct) { - super(struct); LinkedHashMap responseData = new LinkedHashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; @@ -137,22 +121,31 @@ public FetchResponse(Struct struct) { } } this.responseData = responseData; - this.throttleTime = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + } + + @Override + public Struct toStruct(short version) { + return toStruct(version, responseData, throttleTimeMs); } @Override public Send toSend(String dest, RequestHeader requestHeader) { - ResponseHeader responseHeader = new ResponseHeader(requestHeader.correlationId()); + return toSend(toStruct(requestHeader.apiVersion()), throttleTimeMs, dest, requestHeader); + } + + public Send toSend(Struct responseStruct, int throttleTimeMs, String dest, RequestHeader requestHeader) { + Struct responseHeader = new ResponseHeader(requestHeader.correlationId()).toStruct(); // write the total size and the response header ByteBuffer buffer = ByteBuffer.allocate(responseHeader.sizeOf() + 4); - buffer.putInt(responseHeader.sizeOf() + struct.sizeOf()); + buffer.putInt(responseHeader.sizeOf() + responseStruct.sizeOf()); responseHeader.writeTo(buffer); buffer.rewind(); List sends = new ArrayList<>(); sends.add(new ByteBufferSend(dest, buffer)); - addResponseData(dest, sends); + addResponseData(responseStruct, throttleTimeMs, dest, sends); return new MultiSend(dest, sends); } @@ -160,25 +153,20 @@ public LinkedHashMap responseData() { return responseData; } - public int getThrottleTime() { - return this.throttleTime; + public int throttleTimeMs() { + return this.throttleTimeMs; } - public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static FetchResponse parse(ByteBuffer buffer, int version) { + public static FetchResponse parse(ByteBuffer buffer, short version) { return new FetchResponse(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version).read(buffer)); } - private void addResponseData(String dest, List sends) { + private static void addResponseData(Struct struct, int throttleTimeMs, String dest, List sends) { Object[] allTopicData = struct.getArray(RESPONSES_KEY_NAME); if (struct.hasField(THROTTLE_TIME_KEY_NAME)) { - int throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); ByteBuffer buffer = ByteBuffer.allocate(8); - buffer.putInt(throttleTime); + buffer.putInt(throttleTimeMs); buffer.putInt(allTopicData.length); buffer.rewind(); sends.add(new ByteBufferSend(dest, buffer)); @@ -193,7 +181,7 @@ private void addResponseData(String dest, List sends) { addTopicData(dest, sends, (Struct) topicData); } - private void addTopicData(String dest, List sends, Struct topicData) { + private static void addTopicData(String dest, List sends, Struct topicData) { String topic = topicData.getString(TOPIC_KEY_NAME); Object[] allPartitionData = topicData.getArray(PARTITIONS_KEY_NAME); @@ -208,7 +196,7 @@ private void addTopicData(String dest, List sends, Struct topicData) { addPartitionData(dest, sends, (Struct) partitionData); } - private void addPartitionData(String dest, List sends, Struct partitionData) { + private static void addPartitionData(String dest, List sends, Struct partitionData) { Struct header = partitionData.getStruct(PARTITION_HEADER_KEY_NAME); Records records = partitionData.getRecords(RECORD_SET_KEY_NAME); @@ -223,10 +211,8 @@ private void addPartitionData(String dest, List sends, Struct partitionDat sends.add(new RecordsSend(dest, records)); } - private static Struct writeStruct(Struct struct, - int version, - LinkedHashMap responseData, - int throttleTime) { + private static Struct toStruct(short version, LinkedHashMap responseData, int throttleTime) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)); List> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData); List topicArray = new ArrayList<>(); for (FetchRequest.TopicAndPartitionData topicEntry: topicsData) { @@ -255,10 +241,8 @@ private static Struct writeStruct(Struct struct, return struct; } - public static int sizeOf(int version, LinkedHashMap responseData) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)); - writeStruct(struct, version, responseData, 0); - return 4 + struct.sizeOf(); + public static int sizeOf(short version, LinkedHashMap responseData) { + return 4 + toStruct(version, responseData, 0).sizeOf(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java index ed56f3997f7d5..83d6cba27a36d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java @@ -32,8 +32,7 @@ public Builder(String groupId) { } @Override - public GroupCoordinatorRequest build() { - short version = version(); + public GroupCoordinatorRequest build(short version) { return new GroupCoordinatorRequest(this.groupId, version); } @@ -49,14 +48,12 @@ public String toString() { private final String groupId; private GroupCoordinatorRequest(String groupId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.GROUP_COORDINATOR.id, version)), - version); - struct.set(GROUP_ID_KEY_NAME, groupId); + super(version); this.groupId = groupId; } public GroupCoordinatorRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); groupId = struct.getString(GROUP_ID_KEY_NAME); } @@ -76,12 +73,15 @@ public String groupId() { return groupId; } - public static GroupCoordinatorRequest parse(ByteBuffer buffer, int versionId) { + public static GroupCoordinatorRequest parse(ByteBuffer buffer, short versionId) { return new GroupCoordinatorRequest(ProtoUtils.parseRequest(ApiKeys.GROUP_COORDINATOR.id, versionId, buffer), - (short) versionId); + versionId); } - public static GroupCoordinatorRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.GROUP_COORDINATOR.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.GROUP_COORDINATOR.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java index fc3d358668cac..c13cf3c3ab707 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java @@ -16,14 +16,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class GroupCoordinatorResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.GROUP_COORDINATOR.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String COORDINATOR_KEY_NAME = "coordinator"; @@ -45,19 +43,11 @@ public class GroupCoordinatorResponse extends AbstractResponse { private final Node node; public GroupCoordinatorResponse(Errors error, Node node) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); - coordinator.set(NODE_ID_KEY_NAME, node.id()); - coordinator.set(HOST_KEY_NAME, node.host()); - coordinator.set(PORT_KEY_NAME, node.port()); - struct.set(COORDINATOR_KEY_NAME, coordinator); this.error = error; this.node = node; } public GroupCoordinatorResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME); int nodeId = broker.getInt(NODE_ID_KEY_NAME); @@ -74,7 +64,19 @@ public Node node() { return node; } - public static GroupCoordinatorResponse parse(ByteBuffer buffer) { - return new GroupCoordinatorResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.GROUP_COORDINATOR.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); + coordinator.set(NODE_ID_KEY_NAME, node.id()); + coordinator.set(HOST_KEY_NAME, node.host()); + coordinator.set(PORT_KEY_NAME, node.port()); + struct.set(COORDINATOR_KEY_NAME, coordinator); + return struct; + } + + public static GroupCoordinatorResponse parse(ByteBuffer buffer, short version) { + return new GroupCoordinatorResponse(ProtoUtils.parseResponse(ApiKeys.GROUP_COORDINATOR.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 7e79c8a4ff212..444083075c51c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -37,8 +37,8 @@ public Builder(String groupId, int groupGenerationId, String memberId) { } @Override - public HeartbeatRequest build() { - return new HeartbeatRequest(groupId, groupGenerationId, memberId, version()); + public HeartbeatRequest build(short version) { + return new HeartbeatRequest(groupId, groupGenerationId, memberId, version); } @Override @@ -58,18 +58,14 @@ public String toString() { private final String memberId; private HeartbeatRequest(String groupId, int groupGenerationId, String memberId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.HEARTBEAT.id, version)), - version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); + super(version); this.groupId = groupId; this.groupGenerationId = groupGenerationId; this.memberId = memberId; } - public HeartbeatRequest(Struct struct, short versionId) { - super(struct, versionId); + public HeartbeatRequest(Struct struct, short version) { + super(version); groupId = struct.getString(GROUP_ID_KEY_NAME); groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME); memberId = struct.getString(MEMBER_ID_KEY_NAME); @@ -99,11 +95,16 @@ public String memberId() { return memberId; } - public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) { - return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer), (short) versionId); + public static HeartbeatRequest parse(ByteBuffer buffer, short versionId) { + return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer), versionId); } - public static HeartbeatRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.HEARTBEAT.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); + struct.set(MEMBER_ID_KEY_NAME, memberId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index f36dec446f559..4cca846b3f6bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -15,14 +15,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class HeartbeatResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -35,17 +33,13 @@ public class HeartbeatResponse extends AbstractResponse { * REBALANCE_IN_PROGRESS (27) * GROUP_AUTHORIZATION_FAILED (30) */ - private final Errors error; public HeartbeatResponse(Errors error) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); this.error = error; } public HeartbeatResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); } @@ -53,7 +47,14 @@ public Errors error() { return error; } - public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.HEARTBEAT.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + return struct; + } + + public static HeartbeatResponse parse(ByteBuffer buffer, short version) { + return new HeartbeatResponse(ProtoUtils.parseResponse(ApiKeys.HEARTBEAT.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index ad0cdd0f7fb8d..37906a626c74e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -85,8 +85,7 @@ public Builder setRebalanceTimeout(int rebalanceTimeout) { } @Override - public JoinGroupRequest build() { - short version = version(); + public JoinGroupRequest build(short version) { if (version < 1) { rebalanceTimeout = -1; } @@ -112,23 +111,7 @@ public String toString() { private JoinGroupRequest(short version, String groupId, int sessionTimeout, int rebalanceTimeout, String memberId, String protocolType, List groupProtocols) { - super(new Struct(ProtoUtils. - requestSchema(ApiKeys.JOIN_GROUP.id, version)), version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); - if (version >= 1) { - struct.set(REBALANCE_TIMEOUT_KEY_NAME, rebalanceTimeout); - } - struct.set(MEMBER_ID_KEY_NAME, memberId); - struct.set(PROTOCOL_TYPE_KEY_NAME, protocolType); - List groupProtocolsList = new ArrayList<>(groupProtocols.size()); - for (ProtocolMetadata protocol : groupProtocols) { - Struct protocolStruct = struct.instance(GROUP_PROTOCOLS_KEY_NAME); - protocolStruct.set(PROTOCOL_NAME_KEY_NAME, protocol.name); - protocolStruct.set(PROTOCOL_METADATA_KEY_NAME, protocol.metadata); - groupProtocolsList.add(protocolStruct); - } - struct.set(GROUP_PROTOCOLS_KEY_NAME, groupProtocolsList.toArray()); + super(version); this.groupId = groupId; this.sessionTimeout = sessionTimeout; this.rebalanceTimeout = rebalanceTimeout; @@ -138,7 +121,7 @@ private JoinGroupRequest(short version, String groupId, int sessionTimeout, } public JoinGroupRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); groupId = struct.getString(GROUP_ID_KEY_NAME); sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME); @@ -169,7 +152,6 @@ public AbstractResponse getErrorResponse(Throwable e) { case 0: case 1: return new JoinGroupResponse( - versionId, Errors.forException(e), JoinGroupResponse.UNKNOWN_GENERATION_ID, JoinGroupResponse.UNKNOWN_PROTOCOL, @@ -207,12 +189,29 @@ public String protocolType() { return protocolType; } - public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) { - return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer), - (short) versionId); + public static JoinGroupRequest parse(ByteBuffer buffer, short version) { + return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, version, buffer), version); } - public static JoinGroupRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)); + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.JOIN_GROUP.id, version)); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); + if (version >= 1) { + struct.set(REBALANCE_TIMEOUT_KEY_NAME, rebalanceTimeout); + } + struct.set(MEMBER_ID_KEY_NAME, memberId); + struct.set(PROTOCOL_TYPE_KEY_NAME, protocolType); + List groupProtocolsList = new ArrayList<>(groupProtocols.size()); + for (ProtocolMetadata protocol : groupProtocols) { + Struct protocolStruct = struct.instance(GROUP_PROTOCOLS_KEY_NAME); + protocolStruct.set(PROTOCOL_NAME_KEY_NAME, protocol.name); + protocolStruct.set(PROTOCOL_METADATA_KEY_NAME, protocol.metadata); + groupProtocolsList.add(protocolStruct); + } + struct.set(GROUP_PROTOCOLS_KEY_NAME, groupProtocolsList.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index bc9366a91cc11..d2a323b157bf7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -26,8 +25,6 @@ public class JoinGroupResponse extends AbstractResponse { - private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id); - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -67,33 +64,6 @@ public JoinGroupResponse(Errors error, String memberId, String leaderId, Map groupMembers) { - this(CURRENT_VERSION, error, generationId, groupProtocol, memberId, leaderId, groupMembers); - } - - public JoinGroupResponse(int version, - Errors error, - int generationId, - String groupProtocol, - String memberId, - String leaderId, - Map groupMembers) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version))); - - struct.set(ERROR_CODE_KEY_NAME, error.code()); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(GROUP_PROTOCOL_KEY_NAME, groupProtocol); - struct.set(MEMBER_ID_KEY_NAME, memberId); - struct.set(LEADER_ID_KEY_NAME, leaderId); - - List memberArray = new ArrayList<>(); - for (Map.Entry entries: groupMembers.entrySet()) { - Struct memberData = struct.instance(MEMBERS_KEY_NAME); - memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); - memberData.set(MEMBER_METADATA_KEY_NAME, entries.getValue()); - memberArray.add(memberData); - } - struct.set(MEMBERS_KEY_NAME, memberArray.toArray()); - this.error = error; this.generationId = generationId; this.groupProtocol = groupProtocol; @@ -103,7 +73,6 @@ public JoinGroupResponse(int version, } public JoinGroupResponse(Struct struct) { - super(struct); members = new HashMap<>(); for (Object memberDataObj : struct.getArray(MEMBERS_KEY_NAME)) { @@ -147,11 +116,29 @@ public Map members() { return members; } - public static JoinGroupResponse parse(ByteBuffer buffer, int version) { - return new JoinGroupResponse(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version).read(buffer)); + public static JoinGroupResponse parse(ByteBuffer buffer, short version) { + return new JoinGroupResponse(ProtoUtils.parseResponse(ApiKeys.JOIN_GROUP.id, version, buffer)); } - public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version)); + + struct.set(ERROR_CODE_KEY_NAME, error.code()); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(GROUP_PROTOCOL_KEY_NAME, groupProtocol); + struct.set(MEMBER_ID_KEY_NAME, memberId); + struct.set(LEADER_ID_KEY_NAME, leaderId); + + List memberArray = new ArrayList<>(); + for (Map.Entry entries : members.entrySet()) { + Struct memberData = struct.instance(MEMBERS_KEY_NAME); + memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); + memberData.set(MEMBER_METADATA_KEY_NAME, entries.getValue()); + memberArray.add(memberData); + } + struct.set(MEMBERS_KEY_NAME, memberArray.toArray()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index fde184aee18be..c564b437d9d76 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -69,9 +69,8 @@ public Builder(int controllerId, int controllerEpoch, } @Override - public LeaderAndIsrRequest build() { - return new LeaderAndIsrRequest(controllerId, controllerEpoch, partitionStates, - liveLeaders, version()); + public LeaderAndIsrRequest build(short version) { + return new LeaderAndIsrRequest(controllerId, controllerEpoch, partitionStates, liveLeaders, version); } @Override @@ -94,46 +93,15 @@ public String toString() { private LeaderAndIsrRequest(int controllerId, int controllerEpoch, Map partitionStates, Set liveLeaders, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LEADER_AND_ISR.id, version)), - version); - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - - List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { - Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); - TopicPartition topicPartition = entry.getKey(); - partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); - partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); - PartitionState partitionState = entry.getValue(); - partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); - partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); - partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); - partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); - partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); - partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); - partitionStatesData.add(partitionStateData); - } - struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); - - List leadersData = new ArrayList<>(liveLeaders.size()); - for (Node leader : liveLeaders) { - Struct leaderData = struct.instance(LIVE_LEADERS_KEY_NAME); - leaderData.set(END_POINT_ID_KEY_NAME, leader.id()); - leaderData.set(HOST_KEY_NAME, leader.host()); - leaderData.set(PORT_KEY_NAME, leader.port()); - leadersData.add(leaderData); - } - struct.set(LIVE_LEADERS_KEY_NAME, leadersData.toArray()); - + super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; this.liveLeaders = liveLeaders; } - public LeaderAndIsrRequest(Struct struct, short versionId) { - super(struct, versionId); + public LeaderAndIsrRequest(Struct struct, short version) { + super(version); Map partitionStates = new HashMap<>(); for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { @@ -176,6 +144,42 @@ public LeaderAndIsrRequest(Struct struct, short versionId) { this.liveLeaders = leaders; } + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LEADER_AND_ISR.id, version)); + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + + List partitionStatesData = new ArrayList<>(partitionStates.size()); + for (Map.Entry entry : partitionStates.entrySet()) { + Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); + TopicPartition topicPartition = entry.getKey(); + partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); + partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); + PartitionState partitionState = entry.getValue(); + partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); + partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); + partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); + partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); + partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); + partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + partitionStatesData.add(partitionStateData); + } + struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); + + List leadersData = new ArrayList<>(liveLeaders.size()); + for (Node leader : liveLeaders) { + Struct leaderData = struct.instance(LIVE_LEADERS_KEY_NAME); + leaderData.set(END_POINT_ID_KEY_NAME, leader.id()); + leaderData.set(HOST_KEY_NAME, leader.host()); + leaderData.set(PORT_KEY_NAME, leader.port()); + leadersData.add(leaderData); + } + struct.set(LIVE_LEADERS_KEY_NAME, leadersData.toArray()); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { Map responses = new HashMap<>(partitionStates.size()); @@ -209,12 +213,8 @@ public Set liveLeaders() { return liveLeaders; } - public static LeaderAndIsrRequest parse(ByteBuffer buffer, int versionId) { - return new LeaderAndIsrRequest(ProtoUtils.parseRequest(ApiKeys.LEADER_AND_ISR.id, versionId, buffer), - (short) versionId); + public static LeaderAndIsrRequest parse(ByteBuffer buffer, short versionId) { + return new LeaderAndIsrRequest(ProtoUtils.parseRequest(ApiKeys.LEADER_AND_ISR.id, versionId, buffer), versionId); } - public static LeaderAndIsrRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LEADER_AND_ISR.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index 4d0a05dfad6cd..2b02daf2fd277 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,7 +26,6 @@ import java.util.Map; public class LeaderAndIsrResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LEADER_AND_ISR.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String PARTITIONS_KEY_NAME = "partitions"; @@ -45,33 +43,12 @@ public class LeaderAndIsrResponse extends AbstractResponse { private final Map responses; - public LeaderAndIsrResponse(Map responses) { - this(Errors.NONE, responses); - } - public LeaderAndIsrResponse(Errors error, Map responses) { - super(new Struct(CURRENT_SCHEMA)); - - List responseDatas = new ArrayList<>(responses.size()); - for (Map.Entry response : responses.entrySet()) { - Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); - TopicPartition partition = response.getKey(); - partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); - partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); - partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); - responseDatas.add(partitionData); - } - - struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - this.responses = responses; this.error = error; } public LeaderAndIsrResponse(Struct struct) { - super(struct); - responses = new HashMap<>(); for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { Struct responseData = (Struct) responseDataObj; @@ -92,12 +69,27 @@ public Errors error() { return error; } - public static LeaderAndIsrResponse parse(ByteBuffer buffer, int version) { + public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { return new LeaderAndIsrResponse(ProtoUtils.parseResponse(ApiKeys.LEADER_AND_ISR.id, version, buffer)); } - public static LeaderAndIsrResponse parse(ByteBuffer buffer) { - return new LeaderAndIsrResponse(CURRENT_SCHEMA.read(buffer)); - } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LEADER_AND_ISR.id, version)); + + List responseDatas = new ArrayList<>(responses.size()); + for (Map.Entry response : responses.entrySet()) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + TopicPartition partition = response.getKey(); + partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); + partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); + responseDatas.add(partitionData); + } + struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java index 2a7b70e4c2be6..16622e41e0336 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java @@ -33,8 +33,8 @@ public Builder(String groupId, String memberId) { } @Override - public LeaveGroupRequest build() { - return new LeaveGroupRequest(groupId, memberId, version()); + public LeaveGroupRequest build(short version) { + return new LeaveGroupRequest(groupId, memberId, version); } @Override @@ -52,16 +52,13 @@ public String toString() { private final String memberId; private LeaveGroupRequest(String groupId, String memberId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LEAVE_GROUP.id, version)), - version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(MEMBER_ID_KEY_NAME, memberId); + super(version); this.groupId = groupId; this.memberId = memberId; } public LeaveGroupRequest(Struct struct, short version) { - super(struct, version); + super(version); groupId = struct.getString(GROUP_ID_KEY_NAME); memberId = struct.getString(MEMBER_ID_KEY_NAME); } @@ -86,12 +83,15 @@ public String memberId() { return memberId; } - public static LeaveGroupRequest parse(ByteBuffer buffer, int versionId) { - return new LeaveGroupRequest(ProtoUtils.parseRequest(ApiKeys.LEAVE_GROUP.id, versionId, buffer), - (short) versionId); + public static LeaveGroupRequest parse(ByteBuffer buffer, short versionId) { + return new LeaveGroupRequest(ProtoUtils.parseRequest(ApiKeys.LEAVE_GROUP.id, versionId, buffer), versionId); } - public static LeaveGroupRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LEAVE_GROUP.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LEAVE_GROUP.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(MEMBER_ID_KEY_NAME, memberId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index bd1c84d49c148..a28816a5b0ecc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -12,16 +12,15 @@ */ package org.apache.kafka.common.requests; -import java.nio.ByteBuffer; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; +import java.nio.ByteBuffer; + public class LeaveGroupResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LEAVE_GROUP.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -33,16 +32,13 @@ public class LeaveGroupResponse extends AbstractResponse { * UNKNOWN_CONSUMER_ID (25) * GROUP_AUTHORIZATION_FAILED (30) */ - private final Errors error; + public LeaveGroupResponse(Errors error) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); this.error = error; } public LeaveGroupResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); } @@ -50,7 +46,15 @@ public Errors error() { return error; } - public static LeaveGroupResponse parse(ByteBuffer buffer) { - return new LeaveGroupResponse(CURRENT_SCHEMA.read(buffer)); + @Override + public Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LEAVE_GROUP.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + return struct; } + + public static LeaveGroupResponse parse(ByteBuffer buffer, short versionId) { + return new LeaveGroupResponse(ProtoUtils.parseResponse(ApiKeys.LEAVE_GROUP.id, versionId, buffer)); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java index 235f4e4174183..badb527f38a34 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java @@ -27,8 +27,8 @@ public Builder() { } @Override - public ListGroupsRequest build() { - return new ListGroupsRequest(version()); + public ListGroupsRequest build(short version) { + return new ListGroupsRequest(version); } @Override @@ -38,12 +38,11 @@ public String toString() { } public ListGroupsRequest(short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_GROUPS.id, version)), - version); + super(version); } public ListGroupsRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); } @Override @@ -58,12 +57,12 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static ListGroupsRequest parse(ByteBuffer buffer, int versionId) { - return new ListGroupsRequest(ProtoUtils.parseRequest(ApiKeys.LIST_GROUPS.id, versionId, buffer), - (short) versionId); + public static ListGroupsRequest parse(ByteBuffer buffer, short versionId) { + return new ListGroupsRequest(ProtoUtils.parseRequest(ApiKeys.LIST_GROUPS.id, versionId, buffer), versionId); } - public static ListGroupsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LIST_GROUPS.id)); + @Override + protected Struct toStruct() { + return new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_GROUPS.id, version())); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index f4210640e712a..e05a4b1df5817 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -25,8 +24,6 @@ public class ListGroupsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_GROUPS.id); - public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String GROUPS_KEY_NAME = "groups"; public static final String GROUP_ID_KEY_NAME = "group_id"; @@ -43,22 +40,11 @@ public class ListGroupsResponse extends AbstractResponse { private final List groups; public ListGroupsResponse(Errors error, List groups) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - List groupList = new ArrayList<>(); - for (Group group : groups) { - Struct groupStruct = struct.instance(GROUPS_KEY_NAME); - groupStruct.set(GROUP_ID_KEY_NAME, group.groupId); - groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); - groupList.add(groupStruct); - } - struct.set(GROUPS_KEY_NAME, groupList.toArray()); this.error = error; this.groups = groups; } public ListGroupsResponse(Struct struct) { - super(struct); this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); this.groups = new ArrayList<>(); for (Object groupObj : struct.getArray(GROUPS_KEY_NAME)) { @@ -96,12 +82,27 @@ public String protocolType() { } - public static ListGroupsResponse parse(ByteBuffer buffer) { - return new ListGroupsResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_GROUPS.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + List groupList = new ArrayList<>(); + for (Group group : groups) { + Struct groupStruct = struct.instance(GROUPS_KEY_NAME); + groupStruct.set(GROUP_ID_KEY_NAME, group.groupId); + groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); + groupList.add(groupStruct); + } + struct.set(GROUPS_KEY_NAME, groupList.toArray()); + return struct; } public static ListGroupsResponse fromError(Errors error) { return new ListGroupsResponse(error, Collections.emptyList()); } + public static ListGroupsResponse parse(ByteBuffer buffer, short versionId) { + return new ListGroupsResponse(ProtoUtils.parseResponse(ApiKeys.LIST_GROUPS.id, versionId, buffer)); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 6214a5601bdfc..3e2ad7c1571a6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -60,16 +60,21 @@ public class ListOffsetRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { private final int replicaId; + private final short minVersion; private Map offsetData = null; private Map partitionTimestamps = null; - private short minVersion = (short) 0; - public Builder() { - this(CONSUMER_REPLICA_ID); + public static Builder forReplica(short desiredVersion, int replicaId) { + return new Builder((short) 0, desiredVersion, replicaId); } - public Builder(int replicaId) { - super(ApiKeys.LIST_OFFSETS); + public static Builder forConsumer(short minVersion) { + return new Builder(minVersion, null, CONSUMER_REPLICA_ID); + } + + private Builder(short minVersion, Short desiredVersion, int replicaId) { + super(ApiKeys.LIST_OFFSETS, desiredVersion); + this.minVersion = minVersion; this.replicaId = replicaId; } @@ -84,8 +89,7 @@ public Builder setTargetTimes(Map partitionTimestamps) { } @Override - public ListOffsetRequest build() { - short version = version(); + public ListOffsetRequest build(short version) { if (version < minVersion) { throw new UnsupportedVersionException("Cannot create a v" + version + " ListOffsetRequest because " + "we require features supported only in " + minVersion + " or later."); @@ -117,14 +121,6 @@ public ListOffsetRequest build() { return new ListOffsetRequest(replicaId, m, version); } - /** - * Set the minimum version which we will produce for this request. - */ - public Builder setMinVersion(short minVersion) { - this.minVersion = minVersion; - return this; - } - @Override public String toString() { StringBuilder bld = new StringBuilder(); @@ -170,44 +166,15 @@ public String toString() { */ @SuppressWarnings("unchecked") private ListOffsetRequest(int replicaId, Map targetTimes, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_OFFSETS.id, version)), version); - Map> topicsData = - CollectionUtils.groupDataByTopic((Map) targetTimes); - - struct.set(REPLICA_ID_KEY_NAME, replicaId); - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - if (version == 0) { - PartitionData offsetPartitionData = (PartitionData) partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); - partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets); - partitionArray.add(partitionData); - } else { - Long timestamp = (Long) partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(TIMESTAMP_KEY_NAME, timestamp); - partitionArray.add(partitionData); - } - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + super(version); this.replicaId = replicaId; this.offsetData = version == 0 ? (Map) targetTimes : null; this.partitionTimestamps = version == 1 ? (Map) targetTimes : null; this.duplicatePartitions = Collections.emptySet(); } - public ListOffsetRequest(Struct struct, short versionId) { - super(struct, versionId); + public ListOffsetRequest(Struct struct, short version) { + super(version); Set duplicatePartitions = new HashSet<>(); replicaId = struct.getInt(REPLICA_ID_KEY_NAME); offsetData = new HashMap<>(); @@ -236,17 +203,19 @@ public ListOffsetRequest(Struct struct, short versionId) { @Override @SuppressWarnings("deprecation") public AbstractResponse getErrorResponse(Throwable e) { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); short versionId = version(); if (versionId == 0) { for (Map.Entry entry : offsetData.entrySet()) { - ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e), new ArrayList()); + ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData( + Errors.forException(e), Collections.emptyList()); responseData.put(entry.getKey(), partitionResponse); } } else { for (Map.Entry entry : partitionTimestamps.entrySet()) { - ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e), -1L, -1L); + ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData( + Errors.forException(e), -1L, -1L); responseData.put(entry.getKey(), partitionResponse); } } @@ -254,7 +223,7 @@ public AbstractResponse getErrorResponse(Throwable e) { switch (versionId) { case 0: case 1: - return new ListOffsetResponse(responseData, versionId); + return new ListOffsetResponse(responseData); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id))); @@ -278,12 +247,44 @@ public Set duplicatePartitions() { return duplicatePartitions; } - public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) { - return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer), - (short) versionId); + public static ListOffsetRequest parse(ByteBuffer buffer, short versionId) { + return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer), versionId); } - public static ListOffsetRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id)); + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_OFFSETS.id, version)); + + Map targetTimes = partitionTimestamps == null ? offsetData : partitionTimestamps; + Map> topicsData = CollectionUtils.groupDataByTopic(targetTimes); + + struct.set(REPLICA_ID_KEY_NAME, replicaId); + List topicArray = new ArrayList<>(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + if (version == 0) { + PartitionData offsetPartitionData = (PartitionData) partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); + partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets); + partitionArray.add(partitionData); + } else { + Long timestamp = (Long) partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(TIMESTAMP_KEY_NAME, timestamp); + partitionArray.add(partitionData); + } + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index b815a53d6f9c9..cb3bafcc4eb80 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; @@ -35,7 +34,6 @@ public class ListOffsetResponse extends AbstractResponse { public static final long UNKNOWN_TIMESTAMP = -1L; public static final long UNKNOWN_OFFSET = -1L; - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names @@ -61,8 +59,6 @@ public class ListOffsetResponse extends AbstractResponse { private static final String TIMESTAMP_KEY_NAME = "timestamp"; private static final String OFFSET_KEY_NAME = "offset"; - private final Map responseData; - public static final class PartitionData { public final Errors error; // The offsets list is only used in ListOffsetResponse v0. @@ -110,46 +106,17 @@ public String toString() { } } + private final Map responseData; + /** - * Constructor for ListOffsetResponse v0. + * Constructor for all versions. */ - @Deprecated public ListOffsetResponse(Map responseData) { - this(responseData, 0); - } - - public ListOffsetResponse(Map responseData, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version))); - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData offsetPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.error.code()); - if (version == 0) - partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray()); - else { - partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); - partitionData.set(OFFSET_KEY_NAME, offsetPartitionData.offset); - } - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); this.responseData = responseData; } public ListOffsetResponse(Struct struct) { - super(struct); - responseData = new HashMap(); + responseData = new HashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); @@ -178,11 +145,38 @@ public Map responseData() { return responseData; } - public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(CURRENT_SCHEMA.read(buffer)); + public static ListOffsetResponse parse(ByteBuffer buffer, short version) { + return new ListOffsetResponse(ProtoUtils.parseResponse(ApiKeys.LIST_OFFSETS.id, version, buffer)); } - public static ListOffsetResponse parse(ByteBuffer buffer, int version) { - return new ListOffsetResponse(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version).read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version)); + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + + List topicArray = new ArrayList<>(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData offsetPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.error.code()); + if (version == 0) + partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray()); + else { + partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); + partitionData.set(OFFSET_KEY_NAME, offsetPartitionData.offset); + } + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index 16af1b74f6d39..f31315fbc43cc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -51,8 +51,7 @@ public boolean isAllTopics() { } @Override - public MetadataRequest build() { - short version = version(); + public MetadataRequest build(short version) { if (version < 1) { throw new UnsupportedVersionException("MetadataRequest " + "versions older than 1 are not supported."); @@ -79,27 +78,18 @@ public String toString() { private final List topics; - public static MetadataRequest allTopics(short version) { - return new MetadataRequest.Builder(null).setVersion(version).build(); - } - /** * In v0 null is not allowed and and empty list indicates requesting all topics. * Note: modern clients do not support sending v0 requests. * In v1 null indicates requesting all topics, and an empty list indicates requesting no topics. */ public MetadataRequest(List topics, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id, version)), - version); - if (topics == null) - struct.set(TOPICS_KEY_NAME, null); - else - struct.set(TOPICS_KEY_NAME, topics.toArray()); + super(version); this.topics = topics; } public MetadataRequest(Struct struct, short version) { - super(struct, version); + super(version); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); if (topicArray != null) { topics = new ArrayList<>(); @@ -127,7 +117,7 @@ public AbstractResponse getErrorResponse(Throwable e) { case 0: case 1: case 2: - return new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas, versionId); + return new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); @@ -142,12 +132,17 @@ public List topics() { return topics; } - public static MetadataRequest parse(ByteBuffer buffer, int versionId) { - return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer), - (short) versionId); + public static MetadataRequest parse(ByteBuffer buffer, short versionId) { + return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer), versionId); } - public static MetadataRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.METADATA.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id, version())); + if (topics == null) + struct.set(TOPICS_KEY_NAME, null); + else + struct.set(TOPICS_KEY_NAME, topics.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index a8baee52f3a08..268bf8438a12c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -31,7 +31,6 @@ public class MetadataResponse extends AbstractResponse { - private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.METADATA.id); private static final String BROKERS_KEY_NAME = "brokers"; private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata"; @@ -83,78 +82,16 @@ public class MetadataResponse extends AbstractResponse { private final String clusterId; /** - * Constructor for the latest version + * Constructor for all versions. */ public MetadataResponse(List brokers, String clusterId, int controllerId, List topicMetadata) { - this(brokers, clusterId, controllerId, topicMetadata, CURRENT_VERSION); - } - - /** - * Constructor for a specific version - */ - public MetadataResponse(List brokers, String clusterId, int controllerId, List topicMetadata, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version))); this.brokers = brokers; this.controller = getControllerNode(controllerId, brokers); this.topicMetadata = topicMetadata; this.clusterId = clusterId; - - List brokerArray = new ArrayList<>(); - for (Node node : brokers) { - Struct broker = struct.instance(BROKERS_KEY_NAME); - broker.set(NODE_ID_KEY_NAME, node.id()); - broker.set(HOST_KEY_NAME, node.host()); - broker.set(PORT_KEY_NAME, node.port()); - // This field only exists in v1+ - if (broker.hasField(RACK_KEY_NAME)) - broker.set(RACK_KEY_NAME, node.rack()); - brokerArray.add(broker); - } - struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); - - // This field only exists in v1+ - if (struct.hasField(CONTROLLER_ID_KEY_NAME)) - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - - // This field only exists in v2+ - if (struct.hasField(CLUSTER_ID_KEY_NAME)) - struct.set(CLUSTER_ID_KEY_NAME, clusterId); - - List topicMetadataArray = new ArrayList<>(topicMetadata.size()); - for (TopicMetadata metadata : topicMetadata) { - Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, metadata.topic); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code()); - // This field only exists in v1+ - if (topicData.hasField(IS_INTERNAL_KEY_NAME)) - topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal()); - - List partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size()); - for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) { - Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); - partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, partitionMetadata.error.code()); - partitionData.set(PARTITION_KEY_NAME, partitionMetadata.partition); - partitionData.set(LEADER_KEY_NAME, partitionMetadata.leader.id()); - ArrayList replicas = new ArrayList<>(partitionMetadata.replicas.size()); - for (Node node : partitionMetadata.replicas) - replicas.add(node.id()); - partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); - ArrayList isr = new ArrayList<>(partitionMetadata.isr.size()); - for (Node node : partitionMetadata.isr) - isr.add(node.id()); - partitionData.set(ISR_KEY_NAME, isr.toArray()); - partitionMetadataArray.add(partitionData); - - } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionMetadataArray.toArray()); - topicMetadataArray.add(topicData); - } - struct.set(TOPIC_METADATA_KEY_NAME, topicMetadataArray.toArray()); } public MetadataResponse(Struct struct) { - super(struct); - Map brokers = new HashMap<>(); Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME); for (int i = 0; i < brokerStructs.length; i++) { @@ -317,12 +254,8 @@ public String clusterId() { return this.clusterId; } - public static MetadataResponse parse(ByteBuffer buffer) { - return parse(buffer, CURRENT_VERSION); - } - - public static MetadataResponse parse(ByteBuffer buffer, int version) { - return new MetadataResponse(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version).read(buffer)); + public static MetadataResponse parse(ByteBuffer buffer, short version) { + return new MetadataResponse(ProtoUtils.parseResponse(ApiKeys.METADATA.id, version, buffer)); } public static class TopicMetadata { @@ -400,4 +333,60 @@ public List isr() { } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version)); + List brokerArray = new ArrayList<>(); + for (Node node : brokers) { + Struct broker = struct.instance(BROKERS_KEY_NAME); + broker.set(NODE_ID_KEY_NAME, node.id()); + broker.set(HOST_KEY_NAME, node.host()); + broker.set(PORT_KEY_NAME, node.port()); + // This field only exists in v1+ + if (broker.hasField(RACK_KEY_NAME)) + broker.set(RACK_KEY_NAME, node.rack()); + brokerArray.add(broker); + } + struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); + + // This field only exists in v1+ + if (struct.hasField(CONTROLLER_ID_KEY_NAME)) + struct.set(CONTROLLER_ID_KEY_NAME, controller == null ? NO_CONTROLLER_ID : controller.id()); + + // This field only exists in v2+ + if (struct.hasField(CLUSTER_ID_KEY_NAME)) + struct.set(CLUSTER_ID_KEY_NAME, clusterId); + + List topicMetadataArray = new ArrayList<>(topicMetadata.size()); + for (TopicMetadata metadata : topicMetadata) { + Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, metadata.topic); + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code()); + // This field only exists in v1+ + if (topicData.hasField(IS_INTERNAL_KEY_NAME)) + topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal()); + + List partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size()); + for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) { + Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); + partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, partitionMetadata.error.code()); + partitionData.set(PARTITION_KEY_NAME, partitionMetadata.partition); + partitionData.set(LEADER_KEY_NAME, partitionMetadata.leader.id()); + ArrayList replicas = new ArrayList<>(partitionMetadata.replicas.size()); + for (Node node : partitionMetadata.replicas) + replicas.add(node.id()); + partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); + ArrayList isr = new ArrayList<>(partitionMetadata.isr.size()); + for (Node node : partitionMetadata.isr) + isr.add(node.id()); + partitionData.set(ISR_KEY_NAME, isr.toArray()); + partitionMetadataArray.add(partitionData); + + } + topicData.set(PARTITION_METADATA_KEY_NAME, partitionMetadataArray.toArray()); + topicMetadataArray.add(topicData); + } + struct.set(TOPIC_METADATA_KEY_NAME, topicMetadataArray.toArray()); + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 6dd1197422214..bf14f101a71b8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -124,17 +124,15 @@ public Builder setRetentionTime(long retentionTime) { } @Override - public OffsetCommitRequest build() { - short version = version(); + public OffsetCommitRequest build(short version) { switch (version) { case 0: - return new OffsetCommitRequest(groupId, offsetData); + return new OffsetCommitRequest(groupId, DEFAULT_GENERATION_ID, DEFAULT_MEMBER_ID, + DEFAULT_RETENTION_TIME, offsetData, version); case 1: - return new OffsetCommitRequest(groupId, generationId, memberId, - offsetData); case 2: - return new OffsetCommitRequest(groupId, generationId, memberId, - retentionTime, offsetData, version); + long retentionTime = version == 1 ? DEFAULT_RETENTION_TIME : this.retentionTime; + return new OffsetCommitRequest(groupId, generationId, memberId, retentionTime, offsetData, version); default: throw new UnsupportedVersionException("Unsupported version " + version); } @@ -154,57 +152,9 @@ public String toString() { } } - /** - * Constructor for version 0. - * @param groupId - * @param offsetData - */ - private OffsetCommitRequest(String groupId, Map offsetData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)), (short) 0); - - initCommonFields(groupId, offsetData); - this.groupId = groupId; - this.generationId = DEFAULT_GENERATION_ID; - this.memberId = DEFAULT_MEMBER_ID; - this.retentionTime = DEFAULT_RETENTION_TIME; - this.offsetData = offsetData; - } - - /** - * Constructor for version 1. - * @param groupId - * @param generationId - * @param memberId - * @param offsetData - */ - private OffsetCommitRequest(String groupId, int generationId, String memberId, Map offsetData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1)), (short) 1); - - initCommonFields(groupId, offsetData); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); - this.groupId = groupId; - this.generationId = generationId; - this.memberId = memberId; - this.retentionTime = DEFAULT_RETENTION_TIME; - this.offsetData = offsetData; - } - - /** - * Constructor for version 2 and above. - * @param groupId - * @param generationId - * @param memberId - * @param retentionTime - * @param offsetData - */ private OffsetCommitRequest(String groupId, int generationId, String memberId, long retentionTime, Map offsetData, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, version)), version); - initCommonFields(groupId, offsetData); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); - struct.set(RETENTION_TIME_KEY_NAME, retentionTime); + super(version); this.groupId = groupId; this.generationId = generationId; this.memberId = memberId; @@ -212,35 +162,8 @@ private OffsetCommitRequest(String groupId, int generationId, String memberId, l this.offsetData = offsetData; } - private void initCommonFields(String groupId, Map offsetData) { - Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); - - struct.set(GROUP_ID_KEY_NAME, groupId); - List topicArray = new ArrayList(); - - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - // Only for v1 - if (partitionData.hasField(TIMESTAMP_KEY_NAME)) - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - } - public OffsetCommitRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); groupId = struct.getString(GROUP_ID_KEY_NAME); // This field only exists in v1. @@ -283,6 +206,42 @@ public OffsetCommitRequest(Struct struct, short versionId) { } } + @Override + public Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, version)); + struct.set(GROUP_ID_KEY_NAME, groupId); + + Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); + List topicArray = new ArrayList<>(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + // Only for v1 + if (partitionData.hasField(TIMESTAMP_KEY_NAME)) + partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + if (struct.hasField(GENERATION_ID_KEY_NAME)) + struct.set(GENERATION_ID_KEY_NAME, generationId); + if (struct.hasField(MEMBER_ID_KEY_NAME)) + struct.set(MEMBER_ID_KEY_NAME, memberId); + if (struct.hasField(RETENTION_TIME_KEY_NAME)) + struct.set(RETENTION_TIME_KEY_NAME, retentionTime); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { Map responseData = new HashMap<>(); @@ -322,12 +281,8 @@ public Map offsetData() { return offsetData; } - public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) { + public static OffsetCommitRequest parse(ByteBuffer buffer, short versionId) { Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId); - return new OffsetCommitRequest(schema.read(buffer), (short) versionId); - } - - public static OffsetCommitRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id)); + return new OffsetCommitRequest(schema.read(buffer), versionId); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 8a00c6bcc4f3e..b5709e2dada45 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; @@ -28,7 +27,6 @@ public class OffsetCommitResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); private static final String RESPONSES_KEY_NAME = "responses"; // topic level fields @@ -58,15 +56,33 @@ public class OffsetCommitResponse extends AbstractResponse { private final Map responseData; public OffsetCommitResponse(Map responseData) { - super(new Struct(CURRENT_SCHEMA)); + this.responseData = responseData; + } - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + public OffsetCommitResponse(Struct struct) { + responseData = new HashMap<>(); + for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME)); + responseData.put(new TopicPartition(topic, partition), error); + } + } + } + + @Override + public Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_COMMIT.id, version)); - List topicArray = new ArrayList(); + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); for (Map.Entry> entries: topicsData.entrySet()) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); + List partitionArray = new ArrayList<>(); for (Map.Entry partitionEntry : entries.getValue().entrySet()) { Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); @@ -77,34 +93,16 @@ public OffsetCommitResponse(Map responseData) { topicArray.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - this.responseData = responseData; - } - public OffsetCommitResponse(Struct struct) { - super(struct); - responseData = new HashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME)); - responseData.put(new TopicPartition(topic, partition), error); - } - } + return struct; } public Map responseData() { return responseData; } - public static OffsetCommitResponse parse(ByteBuffer buffer, int version) { - Schema schema = ProtoUtils.responseSchema(ApiKeys.OFFSET_COMMIT.id, version); - return new OffsetCommitResponse(schema.read(buffer)); + public static OffsetCommitResponse parse(ByteBuffer buffer, short version) { + return new OffsetCommitResponse(ProtoUtils.parseResponse(ApiKeys.OFFSET_COMMIT.id, version, buffer)); } - public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(CURRENT_SCHEMA.read(buffer)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 43ddf88d02b4f..2a550e508236d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -58,11 +58,11 @@ public boolean isAllTopicPartitions() { } @Override - public OffsetFetchRequest build() { - if (isAllTopicPartitions() && version() < 2) + public OffsetFetchRequest build(short version) { + if (isAllTopicPartitions() && version < 2) throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " + - "v" + version() + ", but we need v2 or newer to request all topic partitions."); - return new OffsetFetchRequest(groupId, partitions, version()); + "v" + version + ", but we need v2 or newer to request all topic partitions."); + return new OffsetFetchRequest(groupId, partitions, version); } @Override @@ -80,39 +80,18 @@ public String toString() { private final List partitions; public static OffsetFetchRequest forAllPartitions(String groupId) { - return new OffsetFetchRequest.Builder(groupId, null).setVersion((short) 2).build(); + return new OffsetFetchRequest.Builder(groupId, null).build((short) 2); } // v0, v1, and v2 have the same fields. private OffsetFetchRequest(String groupId, List partitions, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_FETCH.id, version)), version); - struct.set(GROUP_ID_KEY_NAME, groupId); - if (partitions != null) { - Map> topicsData = CollectionUtils.groupDataByTopic(partitions); - - List topicArray = new ArrayList<>(); - for (Map.Entry> entries : topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList<>(); - for (Integer partitionId : entries.getValue()) { - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionId); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - } else - struct.set(TOPICS_KEY_NAME, null); - + super(version); this.groupId = groupId; this.partitions = partitions; } - public OffsetFetchRequest(Struct struct, short versionId) { - super(struct, versionId); + public OffsetFetchRequest(Struct struct, short version) { + super(version); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); if (topicArray != null) { @@ -150,7 +129,7 @@ public OffsetFetchResponse getErrorResponse(Errors error) { case 0: case 1: case 2: - return new OffsetFetchResponse(error, responsePartitions, versionId); + return new OffsetFetchResponse(error, responsePartitions); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); @@ -170,16 +149,38 @@ public List partitions() { return partitions; } - public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) { - return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer), - (short) versionId); - } - - public static OffsetFetchRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id)); + public static OffsetFetchRequest parse(ByteBuffer buffer, short versionId) { + return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer), versionId); } public boolean isAllPartitions() { return partitions == null; } + + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_FETCH.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + if (partitions != null) { + Map> topicsData = CollectionUtils.groupDataByTopic(partitions); + + List topicArray = new ArrayList<>(); + for (Map.Entry> entries : topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList<>(); + for (Integer partitionId : entries.getValue()) { + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionId); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + } else + struct.set(TOPICS_KEY_NAME, null); + + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 9c1415538c54f..94de4b11a88c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -24,14 +24,11 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; public class OffsetFetchResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); - private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id); private static final String RESPONSES_KEY_NAME = "responses"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -85,51 +82,16 @@ public boolean hasError() { } /** - * Constructor for the latest version. - * @param error Potential coordinator or group level error code - * @param responseData Fetched offset information grouped by topic-partition - */ - public OffsetFetchResponse(Errors error, Map responseData) { - this(error, responseData, CURRENT_VERSION); - } - - /** - * Unified constructor for all versions. + * Constructor for all versions. * @param error Potential coordinator or group level error code (for api version 2 and later) * @param responseData Fetched offset information grouped by topic-partition - * @param version The request API version */ - public OffsetFetchResponse(Errors error, Map responseData, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); - - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - List topicArray = new ArrayList<>(); - for (Map.Entry> entries : topicsData.entrySet()) { - Struct topicData = this.struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : entries.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); - partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.error.code()); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - - this.struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + public OffsetFetchResponse(Errors error, Map responseData) { this.responseData = responseData; this.error = error; - if (version > 1) - this.struct.set(ERROR_CODE_KEY_NAME, this.error.code()); } public OffsetFetchResponse(Struct struct) { - super(struct); Errors topLevelError = Errors.NONE; this.responseData = new HashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { @@ -175,12 +137,37 @@ public Map responseData() { return responseData; } - public static OffsetFetchResponse parse(ByteBuffer buffer, int version) { - Schema schema = ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version); - return new OffsetFetchResponse(schema.read(buffer)); + public static OffsetFetchResponse parse(ByteBuffer buffer, short version) { + return new OffsetFetchResponse(ProtoUtils.parseResponse(ApiKeys.OFFSET_FETCH.id, version, buffer)); } - public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version)); + + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); + for (Map.Entry> entries : topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : entries.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.error.code()); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + + if (version > 1) + struct.set(ERROR_CODE_KEY_NAME, this.error.code()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index bd3ae8ff3b531..df70e20bd6f42 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -55,11 +55,10 @@ public Builder(short acks, int timeout, Map parti } @Override - public ProduceRequest build() { - short version = version(); - if (version < 2) { + public ProduceRequest build(short version) { + if (version < 2) throw new UnsupportedVersionException("ProduceRequest versions older than 2 are not supported."); - } + return new ProduceRequest(version, acks, timeout, partitionRecords); } @@ -80,33 +79,14 @@ public String toString() { private final Map partitionRecords; private ProduceRequest(short version, short acks, int timeout, Map partitionRecords) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, version)), version); - Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); - struct.set(ACKS_KEY_NAME, acks); - struct.set(TIMEOUT_KEY_NAME, timeout); - List topicDatas = new ArrayList<>(recordsByTopic.size()); - for (Map.Entry> entry : recordsByTopic.entrySet()) { - Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : entry.getValue().entrySet()) { - MemoryRecords records = partitionEntry.getValue(); - Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) - .set(PARTITION_KEY_NAME, partitionEntry.getKey()) - .set(RECORD_SET_KEY_NAME, records); - partitionArray.add(part); - } - topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); + super(version); this.acks = acks; this.timeout = timeout; this.partitionRecords = partitionRecords; } public ProduceRequest(Struct struct, short version) { - super(struct, version); + super(version); partitionRecords = new HashMap<>(); for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) { Struct topicData = (Struct) topicDataObj; @@ -122,6 +102,34 @@ public ProduceRequest(Struct struct, short version) { timeout = struct.getInt(TIMEOUT_KEY_NAME); } + /** + * Visible for testing. + */ + @Override + public Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, version())); + Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); + struct.set(ACKS_KEY_NAME, acks); + struct.set(TIMEOUT_KEY_NAME, timeout); + List topicDatas = new ArrayList<>(recordsByTopic.size()); + for (Map.Entry> entry : recordsByTopic.entrySet()) { + Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : entry.getValue().entrySet()) { + MemoryRecords records = partitionEntry.getValue(); + Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) + .set(PARTITION_KEY_NAME, partitionEntry.getKey()) + .set(RECORD_SET_KEY_NAME, records); + partitionArray.add(part); + } + topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); + topicDatas.add(topicData); + } + struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { /* In case the producer doesn't actually want any response */ @@ -137,10 +145,9 @@ public AbstractResponse getErrorResponse(Throwable e) { short versionId = version(); switch (versionId) { case 0: - return new ProduceResponse(responseMap); case 1: case 2: - return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME, versionId); + return new ProduceResponse(responseMap); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id))); @@ -160,15 +167,10 @@ public Map partitionRecords() { } public void clearPartitionRecords() { - struct.clear(); partitionRecords.clear(); } - public static ProduceRequest parse(ByteBuffer buffer, int versionId) { - return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer), (short) versionId); - } - - public static ProduceRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); + public static ProduceRequest parse(ByteBuffer buffer, short versionId) { + return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer), versionId); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 9eaaadf617e15..7a022affc340a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.CollectionUtils; @@ -31,8 +30,7 @@ * This wrapper supports both v0 and v1 of ProduceResponse. */ public class ProduceResponse extends AbstractResponse { - - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names @@ -73,10 +71,7 @@ public class ProduceResponse extends AbstractResponse { * @param responses Produced data grouped by topic-partition */ public ProduceResponse(Map responses) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0))); - initCommonFields(responses); - this.responses = responses; - this.throttleTime = DEFAULT_THROTTLE_TIME; + this(responses, DEFAULT_THROTTLE_TIME); } /** @@ -85,30 +80,14 @@ public ProduceResponse(Map responses) { * @param throttleTime Time in milliseconds the response was throttled */ public ProduceResponse(Map responses, int throttleTime) { - this(responses, throttleTime, ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); - } - - /** - * Constructor for a specific version - * @param responses Produced data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled - * @param version the version of schema to use. - */ - public ProduceResponse(Map responses, int throttleTime, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version))); - initCommonFields(responses); - if (struct.hasField(THROTTLE_TIME_KEY_NAME)) - struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); this.responses = responses; this.throttleTime = throttleTime; } /** - * Constructor from a {@link Struct}. It is the caller's responsibility to pass in a struct with the latest schema. - * @param struct + * Constructor from a {@link Struct}. */ public ProduceResponse(Struct struct) { - super(struct); responses = new HashMap<>(); for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicRespStruct = (Struct) topicResponse; @@ -126,7 +105,10 @@ public ProduceResponse(Struct struct) { this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); } - private void initCommonFields(Map responses) { + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version)); + Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList<>(responseByTopic.size()); for (Map.Entry> entry : responseByTopic.entrySet()) { @@ -140,13 +122,17 @@ private void initCommonFields(Map responses) .set(ERROR_CODE_KEY_NAME, part.error.code()) .set(BASE_OFFSET_KEY_NAME, part.baseOffset); if (partStruct.hasField(LOG_APPEND_TIME_KEY_NAME)) - partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime); + partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime); partitionArray.add(partStruct); } topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); topicDatas.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); + + if (struct.hasField(THROTTLE_TIME_KEY_NAME)) + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); + return struct; } public Map responses() { @@ -187,7 +173,7 @@ public String toString() { } } - public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(CURRENT_SCHEMA.read(buffer)); + public static ProduceResponse parse(ByteBuffer buffer, short version) { + return new ProduceResponse(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java new file mode 100644 index 0000000000000..d2147b3181400 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java @@ -0,0 +1,27 @@ +/** + * 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; + +public class RequestAndSize { + public final AbstractRequest request; + public final int size; + + public RequestAndSize(AbstractRequest request, int size) { + this.request = request; + this.size = size; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index 05b78cbe9e1d7..5e65132449ff3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -35,8 +35,7 @@ public class RequestHeader extends AbstractRequestResponse { private final String clientId; private final int correlationId; - public RequestHeader(Struct header) { - super(header); + public RequestHeader(Struct struct) { apiKey = struct.getShort(API_KEY_FIELD); apiVersion = struct.getShort(API_VERSION_FIELD); clientId = struct.getString(CLIENT_ID_FIELD); @@ -44,17 +43,21 @@ public RequestHeader(Struct header) { } public RequestHeader(short apiKey, short version, String client, int correlation) { - super(new Struct(Protocol.REQUEST_HEADER)); - struct.set(API_KEY_FIELD, apiKey); - struct.set(API_VERSION_FIELD, version); - struct.set(CLIENT_ID_FIELD, client); - struct.set(CORRELATION_ID_FIELD, correlation); this.apiKey = apiKey; this.apiVersion = version; this.clientId = client; this.correlationId = correlation; } + public Struct toStruct() { + Struct struct = new Struct(Protocol.REQUEST_HEADER); + struct.set(API_KEY_FIELD, apiKey); + struct.set(API_VERSION_FIELD, apiVersion); + struct.set(CLIENT_ID_FIELD, clientId); + struct.set(CORRELATION_ID_FIELD, correlationId); + return struct; + } + public short apiKey() { return apiKey; } @@ -71,6 +74,10 @@ public int correlationId() { return correlationId; } + public ResponseHeader toResponseHeader() { + return new ResponseHeader(correlationId); + } + public static RequestHeader parse(ByteBuffer buffer) { return new RequestHeader(Protocol.REQUEST_HEADER.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index e68bd39a7e3c1..04390eaae34eb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -34,17 +34,24 @@ public class ResponseHeader extends AbstractRequestResponse { private final int correlationId; - public ResponseHeader(Struct header) { - super(header); + public ResponseHeader(Struct struct) { correlationId = struct.getInt(CORRELATION_KEY_FIELD); } public ResponseHeader(int correlationId) { - super(new Struct(Protocol.RESPONSE_HEADER)); - struct.set(CORRELATION_KEY_FIELD, correlationId); this.correlationId = correlationId; } + public int sizeOf() { + return toStruct().sizeOf(); + } + + public Struct toStruct() { + Struct struct = new Struct(Protocol.RESPONSE_HEADER); + struct.set(CORRELATION_KEY_FIELD, correlationId); + return struct; + } + public int correlationId() { return correlationId; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java index d244f0af682c1..a1f3f0efd35ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; @@ -40,19 +39,17 @@ */ public class SaslHandshakeRequest extends AbstractRequest { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.SASL_HANDSHAKE.id); public static final String MECHANISM_KEY_NAME = "mechanism"; private final String mechanism; public SaslHandshakeRequest(String mechanism) { - super(new Struct(CURRENT_SCHEMA), ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); - struct.set(MECHANISM_KEY_NAME, mechanism); + super(ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); this.mechanism = mechanism; } - public SaslHandshakeRequest(Struct struct, short versionId) { - super(struct, versionId); + public SaslHandshakeRequest(Struct struct, short version) { + super(version); mechanism = struct.getString(MECHANISM_KEY_NAME); } @@ -73,13 +70,15 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static SaslHandshakeRequest parse(ByteBuffer buffer, int versionId) { - return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer), - (short) versionId); + public static SaslHandshakeRequest parse(ByteBuffer buffer, short versionId) { + return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer), versionId); } - public static SaslHandshakeRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.SASL_HANDSHAKE.id, version())); + struct.set(MECHANISM_KEY_NAME, mechanism); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index f50c5becf4ce3..9d38c6ab21a3f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; @@ -36,8 +35,6 @@ */ public class SaslHandshakeResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SASL_HANDSHAKE.id); - private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String ENABLED_MECHANISMS_KEY_NAME = "enabled_mechanisms"; @@ -50,15 +47,11 @@ public class SaslHandshakeResponse extends AbstractResponse { private final List enabledMechanisms; public SaslHandshakeResponse(Errors error, Collection enabledMechanisms) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray()); this.error = error; this.enabledMechanisms = new ArrayList<>(enabledMechanisms); } public SaslHandshakeResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); Object[] mechanisms = struct.getArray(ENABLED_MECHANISMS_KEY_NAME); ArrayList enabledMechanisms = new ArrayList<>(); @@ -71,15 +64,19 @@ public Errors error() { return error; } - public List enabledMechanisms() { - return enabledMechanisms; + @Override + public Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.SASL_HANDSHAKE.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray()); + return struct; } - public static SaslHandshakeResponse parse(ByteBuffer buffer) { - return new SaslHandshakeResponse(CURRENT_SCHEMA.read(buffer)); + public List enabledMechanisms() { + return enabledMechanisms; } - public static SaslHandshakeResponse parse(ByteBuffer buffer, int version) { + public static SaslHandshakeResponse parse(ByteBuffer buffer, short version) { return new SaslHandshakeResponse(ProtoUtils.parseResponse(ApiKeys.SASL_HANDSHAKE.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index ff2638b01a2bb..91806f18f2b48 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -52,9 +52,9 @@ public Builder(int controllerId, int controllerEpoch, boolean deletePartitions, } @Override - public StopReplicaRequest build() { + public StopReplicaRequest build(short version) { return new StopReplicaRequest(controllerId, controllerEpoch, - deletePartitions, partitions, version()); + deletePartitions, partitions, version); } @Override @@ -77,30 +77,15 @@ public String toString() { private StopReplicaRequest(int controllerId, int controllerEpoch, boolean deletePartitions, Set partitions, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.STOP_REPLICA.id, version)), version); - - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions); - - List partitionDatas = new ArrayList<>(partitions.size()); - for (TopicPartition partition : partitions) { - Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); - partitionData.set(TOPIC_KEY_NAME, partition.topic()); - partitionData.set(PARTITION_KEY_NAME, partition.partition()); - partitionDatas.add(partitionData); - } - - struct.set(PARTITIONS_KEY_NAME, partitionDatas.toArray()); - + super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.deletePartitions = deletePartitions; this.partitions = partitions; } - public StopReplicaRequest(Struct struct, short versionId) { - super(struct, versionId); + public StopReplicaRequest(Struct struct, short version) { + super(version); partitions = new HashSet<>(); for (Object partitionDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { @@ -148,12 +133,27 @@ public Set partitions() { return partitions; } - public static StopReplicaRequest parse(ByteBuffer buffer, int versionId) { - return new StopReplicaRequest(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer), - (short) versionId); + public static StopReplicaRequest parse(ByteBuffer buffer, short versionId) { + return new StopReplicaRequest(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer), versionId); } - public static StopReplicaRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.STOP_REPLICA.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.STOP_REPLICA.id, version())); + + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions); + + List partitionDatas = new ArrayList<>(partitions.size()); + for (TopicPartition partition : partitions) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + partitionData.set(TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITION_KEY_NAME, partition.partition()); + partitionDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, partitionDatas.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index b39fb193c511f..5ae5cc1cab5be 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,7 +26,6 @@ import java.util.Map; public class StopReplicaResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.STOP_REPLICA.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String PARTITIONS_KEY_NAME = "partitions"; @@ -37,41 +35,20 @@ public class StopReplicaResponse extends AbstractResponse { private static final String PARTITIONS_ERROR_CODE_KEY_NAME = "error_code"; private final Map responses; - private final Errors error; /** * Possible error code: * * STALE_CONTROLLER_EPOCH (11) */ - - public StopReplicaResponse(Map responses) { - this(Errors.NONE, responses); - } + private final Errors error; public StopReplicaResponse(Errors error, Map responses) { - super(new Struct(CURRENT_SCHEMA)); - - List responseDatas = new ArrayList<>(responses.size()); - for (Map.Entry response : responses.entrySet()) { - Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); - TopicPartition partition = response.getKey(); - partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); - partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); - partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); - responseDatas.add(partitionData); - } - - struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - this.responses = responses; this.error = error; } public StopReplicaResponse(Struct struct) { - super(struct); - responses = new HashMap<>(); for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { Struct responseData = (Struct) responseDataObj; @@ -92,11 +69,27 @@ public Errors error() { return error; } - public static StopReplicaResponse parse(ByteBuffer buffer, int versionId) { - return new StopReplicaResponse(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer)); + public static StopReplicaResponse parse(ByteBuffer buffer, short versionId) { + return new StopReplicaResponse(ProtoUtils.parseResponse(ApiKeys.STOP_REPLICA.id, versionId, buffer)); } - public static StopReplicaResponse parse(ByteBuffer buffer) { - return new StopReplicaResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.STOP_REPLICA.id, version)); + + List responseDatas = new ArrayList<>(responses.size()); + for (Map.Entry response : responses.entrySet()) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + TopicPartition partition = response.getKey(); + partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); + partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); + responseDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java index 937bf98213d7b..7ad5c9a168d68 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java @@ -51,8 +51,8 @@ public Builder(String groupId, int generationId, String memberId, } @Override - public SyncGroupRequest build() { - return new SyncGroupRequest(groupId, generationId, memberId, groupAssignment, version()); + public SyncGroupRequest build(short version) { + return new SyncGroupRequest(groupId, generationId, memberId, groupAssignment, version); } @Override @@ -75,20 +75,7 @@ public String toString() { private SyncGroupRequest(String groupId, int generationId, String memberId, Map groupAssignment, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.SYNC_GROUP.id, version)), version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); - - List memberArray = new ArrayList<>(); - for (Map.Entry entries: groupAssignment.entrySet()) { - Struct memberData = struct.instance(GROUP_ASSIGNMENT_KEY_NAME); - memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); - memberData.set(MEMBER_ASSIGNMENT_KEY_NAME, entries.getValue()); - memberArray.add(memberData); - } - struct.set(GROUP_ASSIGNMENT_KEY_NAME, memberArray.toArray()); - + super(version); this.groupId = groupId; this.generationId = generationId; this.memberId = memberId; @@ -96,7 +83,7 @@ private SyncGroupRequest(String groupId, int generationId, String memberId, } public SyncGroupRequest(Struct struct, short version) { - super(struct, version); + super(version); this.groupId = struct.getString(GROUP_ID_KEY_NAME); this.generationId = struct.getInt(GENERATION_ID_KEY_NAME); this.memberId = struct.getString(MEMBER_ID_KEY_NAME); @@ -141,12 +128,25 @@ public String memberId() { return memberId; } - public static SyncGroupRequest parse(ByteBuffer buffer, int versionId) { - return new SyncGroupRequest(ProtoUtils.parseRequest(ApiKeys.SYNC_GROUP.id, versionId, buffer), - (short) versionId); + public static SyncGroupRequest parse(ByteBuffer buffer, short versionId) { + return new SyncGroupRequest(ProtoUtils.parseRequest(ApiKeys.SYNC_GROUP.id, versionId, buffer), versionId); } - public static SyncGroupRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.SYNC_GROUP.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.SYNC_GROUP.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(MEMBER_ID_KEY_NAME, memberId); + + List memberArray = new ArrayList<>(); + for (Map.Entry entries: groupAssignment.entrySet()) { + Struct memberData = struct.instance(GROUP_ASSIGNMENT_KEY_NAME); + memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); + memberData.set(MEMBER_ASSIGNMENT_KEY_NAME, entries.getValue()); + memberArray.add(memberData); + } + struct.set(GROUP_ASSIGNMENT_KEY_NAME, memberArray.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index e598975187e89..ff198aa0063b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -19,14 +19,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class SyncGroupResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SYNC_GROUP.id); public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment"; @@ -45,18 +43,11 @@ public class SyncGroupResponse extends AbstractResponse { private final ByteBuffer memberState; public SyncGroupResponse(Errors error, ByteBuffer memberState) { - super(new Struct(CURRENT_SCHEMA)); - - struct.set(ERROR_CODE_KEY_NAME, error.code()); - struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState); - this.error = error; this.memberState = memberState; } public SyncGroupResponse(Struct struct) { - super(struct); - this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); this.memberState = struct.getBytes(MEMBER_ASSIGNMENT_KEY_NAME); } @@ -69,8 +60,16 @@ public ByteBuffer memberAssignment() { return memberState; } - public static SyncGroupResponse parse(ByteBuffer buffer) { - return new SyncGroupResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.SYNC_GROUP.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState); + return struct; + } + + public static SyncGroupResponse parse(ByteBuffer buffer, short version) { + return new SyncGroupResponse(ProtoUtils.parseResponse(ApiKeys.SYNC_GROUP.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index ef680ff3d6904..8dd852da6bee1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -38,10 +38,9 @@ public static class Builder extends AbstractRequest.Builder partitionStates; private final Set liveBrokers; - public Builder(int controllerId, int controllerEpoch, - Map partitionStates, - Set liveBrokers) { - super(ApiKeys.UPDATE_METADATA_KEY); + public Builder(short version, int controllerId, int controllerEpoch, + Map partitionStates, Set liveBrokers) { + super(ApiKeys.UPDATE_METADATA_KEY, version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; @@ -49,8 +48,7 @@ public Builder(int controllerId, int controllerEpoch, } @Override - public UpdateMetadataRequest build() { - short version = version(); + public UpdateMetadataRequest build(short version) { if (version == 0) { for (Broker broker : liveBrokers) { if (broker.endPoints.size() != 1 || broker.endPoints.get(0).securityProtocol != SecurityProtocol.PLAINTEXT) { @@ -148,58 +146,7 @@ public String toString() { private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)), version); - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - - List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { - Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); - TopicPartition topicPartition = entry.getKey(); - partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); - partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); - PartitionState partitionState = entry.getValue(); - partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); - partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); - partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); - partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); - partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); - partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); - partitionStatesData.add(partitionStateData); - } - struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); - - List brokersData = new ArrayList<>(liveBrokers.size()); - for (Broker broker : liveBrokers) { - Struct brokerData = struct.instance(LIVE_BROKERS_KEY_NAME); - brokerData.set(BROKER_ID_KEY_NAME, broker.id); - - if (version == 0) { - EndPoint endPoint = broker.endPoints.get(0); - brokerData.set(HOST_KEY_NAME, endPoint.host); - brokerData.set(PORT_KEY_NAME, endPoint.port); - } else { - List endPointsData = new ArrayList<>(broker.endPoints.size()); - for (EndPoint endPoint : broker.endPoints) { - Struct endPointData = brokerData.instance(ENDPOINTS_KEY_NAME); - endPointData.set(PORT_KEY_NAME, endPoint.port); - endPointData.set(HOST_KEY_NAME, endPoint.host); - endPointData.set(SECURITY_PROTOCOL_TYPE_KEY_NAME, endPoint.securityProtocol.id); - if (version >= 3) - endPointData.set(LISTENER_NAME_KEY_NAME, endPoint.listenerName.value()); - endPointsData.add(endPointData); - - } - brokerData.set(ENDPOINTS_KEY_NAME, endPointsData.toArray()); - if (version >= 2) { - brokerData.set(RACK_KEY_NAME, broker.rack); - } - } - - brokersData.add(brokerData); - } - struct.set(LIVE_BROKERS_KEY_NAME, brokersData.toArray()); - + super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; @@ -207,7 +154,7 @@ private UpdateMetadataRequest(short version, int controllerId, int controllerEpo } public UpdateMetadataRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); Map partitionStates = new HashMap<>(); for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { Struct partitionStateData = (Struct) partitionStateDataObj; @@ -276,6 +223,64 @@ public UpdateMetadataRequest(Struct struct, short versionId) { this.liveBrokers = liveBrokers; } + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)); + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + + List partitionStatesData = new ArrayList<>(partitionStates.size()); + for (Map.Entry entry : partitionStates.entrySet()) { + Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); + TopicPartition topicPartition = entry.getKey(); + partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); + partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); + PartitionState partitionState = entry.getValue(); + partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); + partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); + partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); + partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); + partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); + partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + partitionStatesData.add(partitionStateData); + } + struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); + + List brokersData = new ArrayList<>(liveBrokers.size()); + for (Broker broker : liveBrokers) { + Struct brokerData = struct.instance(LIVE_BROKERS_KEY_NAME); + brokerData.set(BROKER_ID_KEY_NAME, broker.id); + + if (version == 0) { + EndPoint endPoint = broker.endPoints.get(0); + brokerData.set(HOST_KEY_NAME, endPoint.host); + brokerData.set(PORT_KEY_NAME, endPoint.port); + } else { + List endPointsData = new ArrayList<>(broker.endPoints.size()); + for (EndPoint endPoint : broker.endPoints) { + Struct endPointData = brokerData.instance(ENDPOINTS_KEY_NAME); + endPointData.set(PORT_KEY_NAME, endPoint.port); + endPointData.set(HOST_KEY_NAME, endPoint.host); + endPointData.set(SECURITY_PROTOCOL_TYPE_KEY_NAME, endPoint.securityProtocol.id); + if (version >= 3) + endPointData.set(LISTENER_NAME_KEY_NAME, endPoint.listenerName.value()); + endPointsData.add(endPointData); + + } + brokerData.set(ENDPOINTS_KEY_NAME, endPointsData.toArray()); + if (version >= 2) { + brokerData.set(RACK_KEY_NAME, broker.rack); + } + } + + brokersData.add(brokerData); + } + struct.set(LIVE_BROKERS_KEY_NAME, brokersData.toArray()); + + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { short versionId = version(); @@ -302,12 +307,9 @@ public Set liveBrokers() { return liveBrokers; } - public static UpdateMetadataRequest parse(ByteBuffer buffer, int versionId) { + public static UpdateMetadataRequest parse(ByteBuffer buffer, short versionId) { return new UpdateMetadataRequest(ProtoUtils.parseRequest(ApiKeys.UPDATE_METADATA_KEY.id, versionId, buffer), - (short) versionId); + versionId); } - public static UpdateMetadataRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index 865d6c608c5a7..0032fca77a7d4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -16,15 +16,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class UpdateMetadataResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.UPDATE_METADATA_KEY.id); - private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -35,13 +32,10 @@ public class UpdateMetadataResponse extends AbstractResponse { private final Errors error; public UpdateMetadataResponse(Errors error) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); this.error = error; } public UpdateMetadataResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); } @@ -49,12 +43,14 @@ public Errors error() { return error; } - public static UpdateMetadataResponse parse(ByteBuffer buffer) { - return new UpdateMetadataResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static UpdateMetadataResponse parse(ByteBuffer buffer, int version) { + public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) { return new UpdateMetadataResponse(ProtoUtils.parseResponse(ApiKeys.UPDATE_METADATA_KEY.id, version, buffer)); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 7f6b7aaa5993f..88f8959282611 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -308,11 +308,12 @@ private boolean handleKafkaRequest(byte[] requestBytes) throws IOException, Auth if (!Protocol.apiVersionSupported(requestHeader.apiKey(), requestHeader.apiVersion())) { if (apiKey == ApiKeys.API_VERSIONS) - sendKafkaResponse(requestHeader, ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION)); + sendKafkaResponse(ApiVersionsResponse.unsupportedVersionSend(node, requestHeader)); else throw new UnsupportedVersionException("Version " + requestHeader.apiVersion() + " is not supported for apiKey " + apiKey); } else { - AbstractRequest request = AbstractRequest.getRequest(requestHeader.apiKey(), requestHeader.apiVersion(), requestBuffer); + AbstractRequest request = AbstractRequest.getRequest(requestHeader.apiKey(), requestHeader.apiVersion(), + requestBuffer).request; LOG.debug("Handle Kafka request {}", apiKey); switch (apiKey) { @@ -373,7 +374,11 @@ private void handleApiVersionsRequest(RequestHeader requestHeader) throws IOExce } private void sendKafkaResponse(RequestHeader requestHeader, AbstractResponse response) throws IOException { - netOutBuffer = response.toSend(node, requestHeader); + sendKafkaResponse(response.toSend(node, requestHeader)); + } + + private void sendKafkaResponse(Send send) throws IOException { + netOutBuffer = send; flushNetOutBufferAndUpdateInterestOps(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java index ba3863734391b..106a7d47f78b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java @@ -26,14 +26,14 @@ public class CollectionUtils { * @param Partition data type * @return partitioned data */ - public static Map> groupDataByTopic(Map data) { - Map> dataByTopic = new HashMap>(); - for (Map.Entry entry: data.entrySet()) { + public static Map> groupDataByTopic(Map data) { + Map> dataByTopic = new HashMap<>(); + for (Map.Entry entry: data.entrySet()) { String topic = entry.getKey().topic(); int partition = entry.getKey().partition(); Map topicData = dataByTopic.get(topic); if (topicData == null) { - topicData = new HashMap(); + topicData = new HashMap<>(); dataByTopic.put(topic, topicData); } topicData.put(partition, entry.getValue()); @@ -47,12 +47,12 @@ public static Map> groupDataByTopic(Map> groupDataByTopic(List partitions) { - Map> partitionsByTopic = new HashMap>(); + Map> partitionsByTopic = new HashMap<>(); for (TopicPartition tp: partitions) { String topic = tp.topic(); List topicData = partitionsByTopic.get(topic); if (topicData == null) { - topicData = new ArrayList(); + topicData = new ArrayList<>(); partitionsByTopic.put(topic, topicData); } topicData.add(tp.partition()); diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 50ed1319cdac3..7712d3c8577b2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -131,7 +131,8 @@ public void disconnect(String node) { while (iter.hasNext()) { ClientRequest request = iter.next(); if (request.destination().equals(node)) { - responses.add(new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + short version = request.requestBuilder().desiredOrLatestVersion(); + responses.add(new ClientResponse(request.makeHeader(version), request.callback(), request.destination(), request.createdTimeMs(), now, true, null, null)); iter.remove(); } @@ -146,13 +147,11 @@ public void send(ClientRequest request, long now) { FutureResponse futureResp = iterator.next(); if (futureResp.node != null && !request.destination().equals(futureResp.node.idString())) continue; - request.requestBuilder().setVersion(nodeApiVersions.usableVersion( - request.requestBuilder().apiKey())); - AbstractRequest abstractRequest = request.requestBuilder().build(); + short usableVersion = nodeApiVersions.usableVersion(request.requestBuilder().apiKey()); + AbstractRequest abstractRequest = request.requestBuilder().build(usableVersion); if (!futureResp.requestMatcher.matches(abstractRequest)) throw new IllegalStateException("Next in line response did not match expected request"); - - ClientResponse resp = new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + ClientResponse resp = new ClientResponse(request.makeHeader(usableVersion), request.callback(), request.destination(), request.createdTimeMs(), time.milliseconds(), futureResp.disconnected, null, futureResp.responseBody); responses.add(resp); iterator.remove(); @@ -192,7 +191,8 @@ public void respond(AbstractResponse response) { public void respond(AbstractResponse response, boolean disconnected) { ClientRequest request = requests.remove(); - responses.add(new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + short version = request.requestBuilder().desiredOrLatestVersion(); + responses.add(new ClientResponse(request.makeHeader(version), request.callback(), request.destination(), request.createdTimeMs(), time.milliseconds(), disconnected, null, response)); } @@ -206,7 +206,8 @@ public void respondFrom(AbstractResponse response, Node node, boolean disconnect ClientRequest request = iterator.next(); if (request.destination().equals(node.idString())) { iterator.remove(); - responses.add(new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + short version = request.requestBuilder().desiredOrLatestVersion(); + responses.add(new ClientResponse(request.makeHeader(version), request.callback(), request.destination(), request.createdTimeMs(), time.milliseconds(), disconnected, null, response)); return; } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index deaf2cc15e14e..c89cc24886bed 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.requests.AbstractRequestResponse; import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.ProduceRequest; @@ -137,9 +136,10 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { ResponseHeader respHeader = new ResponseHeader(request.correlationId()); Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); resp.set("responses", new Object[0]); - int size = respHeader.sizeOf() + resp.sizeOf(); + Struct responseHeaderStruct = respHeader.toStruct(); + int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); - respHeader.writeTo(buffer); + responseHeaderStruct.writeTo(buffer); resp.writeTo(buffer); buffer.flip(); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); @@ -152,9 +152,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { } private void maybeSetExpectedApiVersionsResponse() { - ResponseHeader responseHeader = new ResponseHeader(0); - ByteBuffer buffer = AbstractRequestResponse.serialize(responseHeader, - ApiVersionsResponse.API_VERSIONS_RESPONSE); + ByteBuffer buffer = ApiVersionsResponse.API_VERSIONS_RESPONSE.serialize((short) 0, new ResponseHeader(0)); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index a355aa18d095d..5c4590b36e716 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1436,7 +1436,7 @@ private ListOffsetResponse listOffsetsResponse(Map offsets partitionData.put(partitionOffset.getKey(), new ListOffsetResponse.PartitionData(error, 1L, partitionOffset.getValue())); } - return new ListOffsetResponse(partitionData, 1); + return new ListOffsetResponse(partitionData); } private FetchResponse fetchResponse(Map fetches) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index fec92519efc56..6a17da801aac9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -886,7 +886,7 @@ private ListOffsetResponse listOffsetResponse(TopicPartition tp, Errors error, l ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error, timestamp, offset); Map allPartitionData = new HashMap<>(); allPartitionData.put(tp, partitionData); - return new ListOffsetResponse(allPartitionData, 1); + return new ListOffsetResponse(allPartitionData); } private FetchResponse fetchResponse(MemoryRecords records, Errors error, long hw, int throttleTime) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 699f6e23c0f4f..d0b9639806bd7 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -30,6 +30,7 @@ import org.junit.Test; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; @@ -51,133 +52,161 @@ public class RequestResponseTest { @Test public void testSerialization() throws Exception { - checkSerialization(createRequestHeader(), null); - checkSerialization(createResponseHeader(), null); - checkSerialization(createGroupCoordinatorRequest()); - checkSerialization(createGroupCoordinatorRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createGroupCoordinatorResponse(), null); - checkSerialization(createControlledShutdownRequest()); - checkSerialization(createControlledShutdownResponse(), null); - checkSerialization(createControlledShutdownRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createFetchRequest(3), 3); - checkSerialization(createFetchRequest(3).getErrorResponse(new UnknownServerException()), 3); - checkSerialization(createFetchResponse(), null); - checkSerialization(createHeartBeatRequest()); - checkSerialization(createHeartBeatRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createHeartBeatResponse(), null); - checkSerialization(createJoinGroupRequest(1), 1); - checkSerialization(createJoinGroupRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createJoinGroupRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createJoinGroupResponse(), null); - checkSerialization(createLeaveGroupRequest()); - checkSerialization(createLeaveGroupRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createLeaveGroupResponse(), null); - checkSerialization(createListGroupsRequest()); - checkSerialization(createListGroupsRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createListGroupsResponse(), null); - checkSerialization(createDescribeGroupRequest()); - checkSerialization(createDescribeGroupRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createDescribeGroupResponse(), null); - checkSerialization(createListOffsetRequest(1), 1); - checkSerialization(createListOffsetRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createListOffsetResponse(1), 1); - checkSerialization(MetadataRequest.allTopics((short) 2), 2); - checkSerialization(createMetadataRequest(1, Arrays.asList("topic1")), 1); - checkSerialization(createMetadataRequest(1, Arrays.asList("topic1")).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createMetadataResponse(2), 2); - checkSerialization(createMetadataRequest(2, Arrays.asList("topic1")).getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createOffsetCommitRequest(2), 2); - checkSerialization(createOffsetCommitRequest(2).getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createOffsetCommitResponse(), null); - checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); - checkSerialization(OffsetFetchRequest.forAllPartitions("group1").getErrorResponse(new NotCoordinatorForGroupException()), 2); - checkSerialization(createOffsetFetchRequest(0)); - checkSerialization(createOffsetFetchRequest(1)); - checkSerialization(createOffsetFetchRequest(2)); - checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); - checkSerialization(createOffsetFetchRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createOffsetFetchRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createOffsetFetchRequest(2).getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createOffsetFetchResponse(), null); - checkSerialization(createProduceRequest()); - checkSerialization(createProduceRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createProduceResponse(), null); - checkSerialization(createStopReplicaRequest(true)); - checkSerialization(createStopReplicaRequest(false)); - checkSerialization(createStopReplicaRequest(true).getErrorResponse(new UnknownServerException()), null); - checkSerialization(createStopReplicaResponse(), null); - checkSerialization(createLeaderAndIsrRequest()); - checkSerialization(createLeaderAndIsrRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createLeaderAndIsrResponse(), null); - checkSerialization(createSaslHandshakeRequest()); - checkSerialization(createSaslHandshakeRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createSaslHandshakeResponse(), null); - checkSerialization(createApiVersionRequest()); - checkSerialization(createApiVersionRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createApiVersionResponse(), null); - checkSerialization(createCreateTopicRequest(0), 0); - checkSerialization(createCreateTopicRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createCreateTopicResponse(0), 0); - checkSerialization(createCreateTopicRequest(1), 1); - checkSerialization(createCreateTopicRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createCreateTopicResponse(1), 1); - checkSerialization(createDeleteTopicsRequest()); - checkSerialization(createDeleteTopicsRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createDeleteTopicsResponse(), null); + checkRequest(createGroupCoordinatorRequest()); + checkErrorResponse(createGroupCoordinatorRequest(), new UnknownServerException()); + checkResponse(createGroupCoordinatorResponse(), 0); + checkRequest(createControlledShutdownRequest()); + checkResponse(createControlledShutdownResponse(), 1); + checkErrorResponse(createControlledShutdownRequest(), new UnknownServerException()); + checkRequest(createFetchRequest(3)); + checkErrorResponse(createFetchRequest(3), new UnknownServerException()); + checkResponse(createFetchResponse(), 0); + checkRequest(createHeartBeatRequest()); + checkErrorResponse(createHeartBeatRequest(), new UnknownServerException()); + checkResponse(createHeartBeatResponse(), 0); + checkRequest(createJoinGroupRequest(1)); + checkErrorResponse(createJoinGroupRequest(0), new UnknownServerException()); + checkErrorResponse(createJoinGroupRequest(1), new UnknownServerException()); + checkResponse(createJoinGroupResponse(), 0); + checkRequest(createLeaveGroupRequest()); + checkErrorResponse(createLeaveGroupRequest(), new UnknownServerException()); + checkResponse(createLeaveGroupResponse(), 0); + checkRequest(createListGroupsRequest()); + checkErrorResponse(createListGroupsRequest(), new UnknownServerException()); + checkResponse(createListGroupsResponse(), 0); + checkRequest(createDescribeGroupRequest()); + checkErrorResponse(createDescribeGroupRequest(), new UnknownServerException()); + checkResponse(createDescribeGroupResponse(), 0); + checkRequest(createListOffsetRequest(1)); + checkErrorResponse(createListOffsetRequest(1), new UnknownServerException()); + checkResponse(createListOffsetResponse(1), 1); + checkRequest(MetadataRequest.Builder.allTopics().build((short) 2)); + checkRequest(createMetadataRequest(1, Arrays.asList("topic1"))); + checkErrorResponse(createMetadataRequest(1, Arrays.asList("topic1")), new UnknownServerException()); + checkResponse(createMetadataResponse(), 2); + checkErrorResponse(createMetadataRequest(2, Arrays.asList("topic1")), new UnknownServerException()); + checkRequest(createOffsetCommitRequest(2)); + checkErrorResponse(createOffsetCommitRequest(2), new UnknownServerException()); + checkResponse(createOffsetCommitResponse(), 0); + checkRequest(OffsetFetchRequest.forAllPartitions("group1")); + checkErrorResponse(OffsetFetchRequest.forAllPartitions("group1"), new NotCoordinatorForGroupException()); + checkRequest(createOffsetFetchRequest(0)); + checkRequest(createOffsetFetchRequest(1)); + checkRequest(createOffsetFetchRequest(2)); + checkRequest(OffsetFetchRequest.forAllPartitions("group1")); + checkErrorResponse(createOffsetFetchRequest(0), new UnknownServerException()); + checkErrorResponse(createOffsetFetchRequest(1), new UnknownServerException()); + checkErrorResponse(createOffsetFetchRequest(2), new UnknownServerException()); + checkResponse(createOffsetFetchResponse(), 0); + checkRequest(createProduceRequest()); + checkErrorResponse(createProduceRequest(), new UnknownServerException()); + checkResponse(createProduceResponse(), 2); + checkRequest(createStopReplicaRequest(true)); + checkRequest(createStopReplicaRequest(false)); + checkErrorResponse(createStopReplicaRequest(true), new UnknownServerException()); + checkResponse(createStopReplicaResponse(), 0); + checkRequest(createLeaderAndIsrRequest()); + checkErrorResponse(createLeaderAndIsrRequest(), new UnknownServerException()); + checkResponse(createLeaderAndIsrResponse(), 0); + checkRequest(createSaslHandshakeRequest()); + checkErrorResponse(createSaslHandshakeRequest(), new UnknownServerException()); + checkResponse(createSaslHandshakeResponse(), 0); + checkRequest(createApiVersionRequest()); + checkErrorResponse(createApiVersionRequest(), new UnknownServerException()); + checkResponse(createApiVersionResponse(), 0); + checkRequest(createCreateTopicRequest(0)); + checkErrorResponse(createCreateTopicRequest(0), new UnknownServerException()); + checkResponse(createCreateTopicResponse(), 0); + checkRequest(createCreateTopicRequest(1)); + checkErrorResponse(createCreateTopicRequest(1), new UnknownServerException()); + checkResponse(createCreateTopicResponse(), 1); + checkRequest(createDeleteTopicsRequest()); + checkErrorResponse(createDeleteTopicsRequest(), new UnknownServerException()); + checkResponse(createDeleteTopicsResponse(), 0); checkOlderFetchVersions(); - checkSerialization(createMetadataResponse(0), 0); - checkSerialization(createMetadataResponse(1), 1); - checkSerialization(createMetadataRequest(1, Arrays.asList("topic1")).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createOffsetCommitRequest(0), 0); - checkSerialization(createOffsetCommitRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createOffsetCommitRequest(1), 1); - checkSerialization(createOffsetCommitRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createJoinGroupRequest(0), 0); - checkSerialization(createUpdateMetadataRequest(0, null), 0); - checkSerialization(createUpdateMetadataRequest(0, null).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createUpdateMetadataRequest(1, null), 1); - checkSerialization(createUpdateMetadataRequest(1, "rack1"), 1); - checkSerialization(createUpdateMetadataRequest(1, null).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createUpdateMetadataRequest(2, "rack1"), 2); - checkSerialization(createUpdateMetadataRequest(2, null), 2); - checkSerialization(createUpdateMetadataRequest(2, "rack1").getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createUpdateMetadataRequest(3, "rack1")); - checkSerialization(createUpdateMetadataRequest(3, null)); - checkSerialization(createUpdateMetadataRequest(3, "rack1").getErrorResponse(new UnknownServerException()), 3); - checkSerialization(createUpdateMetadataResponse(), null); - checkSerialization(createListOffsetRequest(0), 0); - checkSerialization(createListOffsetRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createListOffsetResponse(0), 0); + checkResponse(createMetadataResponse(), 0); + checkResponse(createMetadataResponse(), 1); + checkErrorResponse(createMetadataRequest(1, Arrays.asList("topic1")), new UnknownServerException()); + checkRequest(createOffsetCommitRequest(0)); + checkErrorResponse(createOffsetCommitRequest(0), new UnknownServerException()); + checkRequest(createOffsetCommitRequest(1)); + checkErrorResponse(createOffsetCommitRequest(1), new UnknownServerException()); + checkRequest(createJoinGroupRequest(0)); + checkRequest(createUpdateMetadataRequest(0, null)); + checkErrorResponse(createUpdateMetadataRequest(0, null), new UnknownServerException()); + checkRequest(createUpdateMetadataRequest(1, null)); + checkRequest(createUpdateMetadataRequest(1, "rack1")); + checkErrorResponse(createUpdateMetadataRequest(1, null), new UnknownServerException()); + checkRequest(createUpdateMetadataRequest(2, "rack1")); + checkRequest(createUpdateMetadataRequest(2, null)); + checkErrorResponse(createUpdateMetadataRequest(2, "rack1"), new UnknownServerException()); + checkRequest(createUpdateMetadataRequest(3, "rack1")); + checkRequest(createUpdateMetadataRequest(3, null)); + checkErrorResponse(createUpdateMetadataRequest(3, "rack1"), new UnknownServerException()); + checkResponse(createUpdateMetadataResponse(), 0); + checkRequest(createListOffsetRequest(0)); + checkErrorResponse(createListOffsetRequest(0), new UnknownServerException()); + checkResponse(createListOffsetResponse(0), 0); + } + + @Test + public void testRequestHeader() { + RequestHeader header = createRequestHeader(); + ByteBuffer buffer = toBuffer(header.toStruct()); + RequestHeader deserialized = RequestHeader.parse(buffer); + assertEquals(header.apiVersion(), deserialized.apiVersion()); + assertEquals(header.apiKey(), deserialized.apiKey()); + assertEquals(header.clientId(), deserialized.clientId()); + assertEquals(header.correlationId(), deserialized.correlationId()); + } + + @Test + public void testResponseHeader() { + ResponseHeader header = createResponseHeader(); + ByteBuffer buffer = toBuffer(header.toStruct()); + ResponseHeader deserialized = ResponseHeader.parse(buffer); + assertEquals(header.correlationId(), deserialized.correlationId()); } private void checkOlderFetchVersions() throws Exception { int latestVersion = ProtoUtils.latestVersion(ApiKeys.FETCH.id); for (int i = 0; i < latestVersion; ++i) { - checkSerialization(createFetchRequest(i).getErrorResponse(new UnknownServerException()), i); - checkSerialization(createFetchRequest(i), i); + checkErrorResponse(createFetchRequest(i), new UnknownServerException()); + checkRequest(createFetchRequest(i)); } } - private void checkSerialization(AbstractRequest req) throws Exception { - checkSerialization(req, Integer.valueOf(req.version())); + private void checkErrorResponse(AbstractRequest req, Throwable e) throws Exception { + checkResponse(req.getErrorResponse(e), req.version()); + } + + private void checkRequest(AbstractRequest req) throws Exception { + // Check that we can serialize, deserialize and serialize again + // We don't check for equality or hashCode because it is likely to fail for any request containing a HashMap + Struct struct = req.toStruct(); + AbstractRequest deserialized = (AbstractRequest) deserialize(req, struct, req.version()); + deserialized.toStruct(); } - private void checkSerialization(AbstractRequestResponse req, Integer version) throws Exception { - ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); - req.writeTo(buffer); + private void checkResponse(AbstractResponse response, int version) throws Exception { + // Check that we can serialize, deserialize and serialize again + // We don't check for equality or hashCode because it is likely to fail for any response containing a HashMap + Struct struct = response.toStruct((short) version); + AbstractResponse deserialized = (AbstractResponse) deserialize(response, struct, (short) version); + deserialized.toStruct((short) version); + } + + private AbstractRequestResponse deserialize(AbstractRequestResponse req, Struct struct, short version) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { + ByteBuffer buffer = toBuffer(struct); + Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class, Short.TYPE); + return (AbstractRequestResponse) deserializer.invoke(null, buffer, version); + } + + private ByteBuffer toBuffer(Struct struct) { + ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf()); + struct.writeTo(buffer); buffer.rewind(); - AbstractRequestResponse deserialized; - if (version == null) { - Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class); - deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer); - } else { - Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class, Integer.TYPE); - deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer, version); - } - assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + - "(version " + version + ") should be the same.", req, deserialized); - assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should have the same hashcode.", - req.hashCode(), deserialized.hashCode()); + return buffer; } @Test @@ -186,14 +215,17 @@ public void produceResponseVersionTest() { responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, Record.NO_TIMESTAMP)); ProduceResponse v0Response = new ProduceResponse(responseData); - ProduceResponse v1Response = new ProduceResponse(responseData, 10, 1); - ProduceResponse v2Response = new ProduceResponse(responseData, 10, 2); + ProduceResponse v1Response = new ProduceResponse(responseData, 10); + ProduceResponse v2Response = new ProduceResponse(responseData, 10); assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); assertEquals("Throttle time must be 10", 10, v2Response.getThrottleTime()); - assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), v0Response.toStruct().schema()); - assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), v1Response.toStruct().schema()); - assertEquals("Should use schema version 2", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 2), v2Response.toStruct().schema()); + assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), + v0Response.toStruct((short) 0).schema()); + assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), + v1Response.toStruct((short) 1).schema()); + assertEquals("Should use schema version 2", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 2), + v2Response.toStruct((short) 2).schema()); assertEquals("Response data does not match", responseData, v0Response.responses()); assertEquals("Response data does not match", responseData, v1Response.responses()); assertEquals("Response data does not match", responseData, v2Response.responses()); @@ -206,12 +238,14 @@ public void fetchResponseVersionTest() { MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10)); responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE, 1000000, records)); - FetchResponse v0Response = new FetchResponse(0, responseData, 0); - FetchResponse v1Response = new FetchResponse(1, responseData, 10); - assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); - assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); - assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0), v0Response.toStruct().schema()); - assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 1), v1Response.toStruct().schema()); + FetchResponse v0Response = new FetchResponse(responseData, 0); + FetchResponse v1Response = new FetchResponse(responseData, 10); + assertEquals("Throttle time must be zero", 0, v0Response.throttleTimeMs()); + assertEquals("Throttle time must be 10", 10, v1Response.throttleTimeMs()); + assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0), + v0Response.toStruct((short) 0).schema()); + assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 1), + v1Response.toStruct((short) 1).schema()); assertEquals("Response data does not match", responseData, v0Response.responseData()); assertEquals("Response data does not match", responseData, v1Response.responseData()); } @@ -239,19 +273,18 @@ public void verifyFetchResponseFullWrite() throws Exception { // read the body Struct responseBody = ProtoUtils.responseSchema(ApiKeys.FETCH.id, header.apiVersion()).read(buf); - FetchResponse parsedResponse = new FetchResponse(responseBody); - assertEquals(parsedResponse, fetchResponse); + assertEquals(fetchResponse.toStruct(header.apiVersion()), responseBody); - assertEquals(size, responseHeader.sizeOf() + parsedResponse.sizeOf()); + assertEquals(size, responseHeader.sizeOf() + responseBody.sizeOf()); } @Test public void testControlledShutdownResponse() { ControlledShutdownResponse response = createControlledShutdownResponse(); - ByteBuffer buffer = ByteBuffer.allocate(response.sizeOf()); - response.writeTo(buffer); - buffer.rewind(); - ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer); + short version = ProtoUtils.latestVersion(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); + Struct struct = response.toStruct(version); + ByteBuffer buffer = toBuffer(struct); + ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer, version); assertEquals(response.error(), deserialized.error()); assertEquals(response.partitionsRemaining(), deserialized.partitionsRemaining()); } @@ -259,9 +292,8 @@ public void testControlledShutdownResponse() { @Test public void testRequestHeaderWithNullClientId() { RequestHeader header = new RequestHeader((short) 10, (short) 1, null, 10); - ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf()); - header.writeTo(buffer); - buffer.rewind(); + Struct headerStruct = header.toStruct(); + ByteBuffer buffer = toBuffer(headerStruct); RequestHeader deserialized = RequestHeader.parse(buffer); assertEquals(header.apiKey(), deserialized.apiKey()); assertEquals(header.apiVersion(), deserialized.apiVersion()); @@ -294,8 +326,7 @@ private FetchRequest createFetchRequest(int version) { LinkedHashMap fetchData = new LinkedHashMap<>(); fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); - return new FetchRequest.Builder(100, 100000, fetchData).setMaxBytes(1000). - setVersion((short) version).build(); + return FetchRequest.Builder.forConsumer(100, 100000, fetchData).setMaxBytes(1000).build((short) version); } private FetchResponse createFetchResponse() { @@ -320,7 +351,7 @@ private JoinGroupRequest createJoinGroupRequest(int version) { protocols.add(new JoinGroupRequest.ProtocolMetadata("consumer-range", metadata)); if (version == 0) { return new JoinGroupRequest.Builder("group1", 30000, "consumer1", "consumer", protocols). - setVersion((short) version).build(); + build((short) version); } else { return new JoinGroupRequest.Builder("group1", 10000, "consumer1", "consumer", protocols). setRebalanceTimeout(60000).build(); @@ -372,11 +403,11 @@ private ListOffsetRequest createListOffsetRequest(int version) { Map offsetData = Collections.singletonMap( new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); - return new ListOffsetRequest.Builder().setOffsetData(offsetData).setVersion((short) version).build(); + return ListOffsetRequest.Builder.forConsumer((short) 0).setOffsetData(offsetData).build((short) version); } else if (version == 1) { Map offsetData = Collections.singletonMap( new TopicPartition("test", 0), 1000000L); - return new ListOffsetRequest.Builder().setTargetTimes(offsetData).setVersion((short) version).build(); + return ListOffsetRequest.Builder.forConsumer((short) 1).setTargetTimes(offsetData).build((short) version); } else { throw new IllegalArgumentException("Illegal ListOffsetRequest version " + version); } @@ -386,24 +417,24 @@ private ListOffsetRequest createListOffsetRequest(int version) { private ListOffsetResponse createListOffsetResponse(int version) { if (version == 0) { Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE, Arrays.asList(100L))); + responseData.put(new TopicPartition("test", 0), + new ListOffsetResponse.PartitionData(Errors.NONE, Arrays.asList(100L))); return new ListOffsetResponse(responseData); } else if (version == 1) { Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L)); - return new ListOffsetResponse(responseData, 1); + responseData.put(new TopicPartition("test", 0), + new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L)); + return new ListOffsetResponse(responseData); } else { throw new IllegalArgumentException("Illegal ListOffsetResponse version " + version); } } private MetadataRequest createMetadataRequest(int version, List topics) { - return new MetadataRequest.Builder(topics). - setVersion((short) version). - build(); + return new MetadataRequest.Builder(topics).build((short) version); } - private MetadataResponse createMetadataResponse(int version) { + private MetadataResponse createMetadataResponse() { Node node = new Node(1, "host1", 1001); List replicas = Arrays.asList(node); List isr = Arrays.asList(node); @@ -414,7 +445,7 @@ private MetadataResponse createMetadataResponse(int version) { allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false, Collections.emptyList())); - return new MetadataResponse(Arrays.asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata, version); + return new MetadataResponse(Arrays.asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata); } private OffsetCommitRequest createOffsetCommitRequest(int version) { @@ -425,8 +456,7 @@ private OffsetCommitRequest createOffsetCommitRequest(int version) { .setGenerationId(100) .setMemberId("consumer1") .setRetentionTime(1000000) - .setVersion((short) version) - .build(); + .build((short) version); } private OffsetCommitResponse createOffsetCommitResponse() { @@ -437,8 +467,7 @@ private OffsetCommitResponse createOffsetCommitResponse() { private OffsetFetchRequest createOffsetFetchRequest(int version) { return new OffsetFetchRequest.Builder("group1", singletonList(new TopicPartition("test11", 1))) - .setVersion((short) version) - .build(); + .build((short) version); } private OffsetFetchResponse createOffsetFetchResponse() { @@ -540,8 +569,8 @@ private UpdateMetadataRequest createUpdateMetadataRequest(int version, String ra new UpdateMetadataRequest.Broker(0, endPoints1, rack), new UpdateMetadataRequest.Broker(1, endPoints2, rack) )); - return new UpdateMetadataRequest.Builder(1, 10, partitionStates, liveBrokers). - setVersion((short) version).build(); + return new UpdateMetadataRequest.Builder((short) version, 1, 10, partitionStates, + liveBrokers).build(); } private UpdateMetadataResponse createUpdateMetadataResponse() { @@ -584,14 +613,14 @@ private CreateTopicsRequest createCreateTopicRequest(int version, boolean valida Map request = new HashMap<>(); request.put("my_t1", request1); request.put("my_t2", request2); - return new CreateTopicsRequest.Builder(request, 0, validateOnly).setVersion((short) version).build(); + return new CreateTopicsRequest.Builder(request, 0, validateOnly).build((short) version); } - private CreateTopicsResponse createCreateTopicResponse(int version) { + private CreateTopicsResponse createCreateTopicResponse() { Map errors = new HashMap<>(); errors.put("t1", new CreateTopicsResponse.Error(Errors.INVALID_TOPIC_EXCEPTION, null)); errors.put("t2", new CreateTopicsResponse.Error(Errors.LEADER_NOT_AVAILABLE, "Leader with id 5 is not available.")); - return new CreateTopicsResponse(errors, (short) version); + return new CreateTopicsResponse(errors); } private DeleteTopicsRequest createDeleteTopicsRequest() { diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 76fb9b31836cc..3a9e0cec0d646 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -392,7 +392,7 @@ public void testApiVersionsRequestWithUnsupportedVersion() throws Exception { selector.send(request.toSend(node, header)); ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer); - ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer); + ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, (short) 0); assertEquals(Errors.UNSUPPORTED_VERSION, response.error()); // Send ApiVersionsRequest with a supported version. This should succeed. diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 97da1f5d84a7c..f0498212eb855 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -205,7 +205,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, responseData.put(new TopicPartition(topic, partition), new JFetchResponse.PartitionData(Errors.forException(e), -1, MemoryRecords.EMPTY)) } - val errorResponse = new JFetchResponse(versionId, responseData, 0) + val errorResponse = new JFetchResponse(responseData, 0) // Magic value does not matter here because the message set is empty requestChannel.sendResponse(new RequestChannel.Response(request, errorResponse)) } diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala deleted file mode 100644 index 3783c29ccbe70..0000000000000 --- a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala +++ /dev/null @@ -1,56 +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 kafka.api - -import java.nio.ByteBuffer - -import kafka.api.ApiUtils._ -import org.apache.kafka.common.requests.AbstractResponse - -private[kafka] abstract class GenericRequestAndHeader(val versionId: Short, - val correlationId: Int, - val clientId: String, - val body: AbstractResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - 2 /* version id */ + - 4 /* correlation id */ + - (2 + clientId.length) /* client id */ + - body.sizeOf() - } - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; Version: " + versionId) - strBuffer.append("; CorrelationId: " + correlationId) - strBuffer.append("; ClientId: " + clientId) - strBuffer.append("; Body: " + body.toString) - strBuffer.toString() - } -} diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala deleted file mode 100644 index be0c080fbe15e..0000000000000 --- a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala +++ /dev/null @@ -1,47 +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 kafka.api - -import java.nio.ByteBuffer - -import org.apache.kafka.common.requests.AbstractResponse - -private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int, - val body: AbstractResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - 4 /* correlation id */ + - body.sizeOf() - } - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; CorrelationId: " + correlationId) - strBuffer.append("; Body: " + body.toString) - strBuffer.toString() - } -} diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index a87cdc90b3fed..bd483887bc47f 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -129,7 +129,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - if(request.body.asInstanceOf[org.apache.kafka.common.requests.ProduceRequest].acks == 0) { + if (request.body[org.apache.kafka.common.requests.ProduceRequest].acks == 0) { requestChannel.closeConnection(request.processor, request) } else { diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index d8e6a9504d6f6..a2fee6b6b4e1c 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -364,9 +364,9 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging partitionStateInfo.allReplicas.map(Integer.valueOf).asJava) topicPartition -> partitionState } - val leaderAndIsrRequest = new LeaderAndIsrRequest. - Builder(controllerId, controllerEpoch, partitionStates.asJava, leaders.asJava) - controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest, null) + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(controllerId, controllerEpoch, partitionStates.asJava, + leaders.asJava) + controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest) } leaderAndIsrRequestMap.clear() @@ -405,9 +405,8 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) } } - new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, liveBrokers.asJava). - setVersion(version) + new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, + liveBrokers.asJava) } updateMetadataRequestBrokerSet.foreach { broker => diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index c063801edfc09..a5d71602eaeef 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -19,7 +19,7 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer -import java.util.HashMap +import java.util.Collections import java.util.concurrent._ import com.yammer.metrics.core.Gauge @@ -37,16 +37,18 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Time import org.apache.log4j.Logger +import scala.reflect.{classTag, ClassTag} + object RequestChannel extends Logging { val AllDone = Request(processor = 1, connectionId = "2", Session(KafkaPrincipal.ANONYMOUS, InetAddress.getLocalHost), - buffer = getShutdownReceive, startTimeMs = 0, listenerName = new ListenerName(""), + buffer = shutdownReceive, startTimeMs = 0, listenerName = new ListenerName(""), securityProtocol = SecurityProtocol.PLAINTEXT) private val requestLogger = Logger.getLogger("kafka.request.logger") - private def getShutdownReceive = { - val emptyProduceRequest = new ProduceRequest.Builder(0, 0, new HashMap[TopicPartition, MemoryRecords]()).build() + private def shutdownReceive: ByteBuffer = { + val emptyProduceRequest = new ProduceRequest.Builder(0, 0, Collections.emptyMap[TopicPartition, MemoryRecords]).build() val emptyRequestHeader = new RequestHeader(ApiKeys.PRODUCE.id, emptyProduceRequest.version, "", 0) - AbstractRequestResponse.serialize(emptyRequestHeader, emptyProduceRequest) + emptyProduceRequest.serialize(emptyRequestHeader) } case class Session(principal: KafkaPrincipal, clientAddress: InetAddress) { @@ -84,12 +86,13 @@ object RequestChannel extends Logging { } } else null - val body: AbstractRequest = + val bodyAndSize: RequestAndSize = if (requestObj == null) try { // For unsupported version of ApiVersionsRequest, create a dummy request to enable an error response to be returned later - if (header.apiKey == ApiKeys.API_VERSIONS.id && !Protocol.apiVersionSupported(header.apiKey, header.apiVersion)) - new ApiVersionsRequest.Builder().build() + if (header.apiKey == ApiKeys.API_VERSIONS.id && !Protocol.apiVersionSupported(header.apiKey, header.apiVersion)) { + new RequestAndSize(new ApiVersionsRequest.Builder().build(), 0) + } else AbstractRequest.getRequest(header.apiKey, header.apiVersion, buffer) } catch { @@ -108,6 +111,14 @@ object RequestChannel extends Logging { s"$header -- $body" } + def body[T <: AbstractRequest : ClassTag] = { + bodyAndSize.request match { + case r: T => r + case r => + throw new ClassCastException(s"Expected request with type ${classTag[T].runtimeClass}, but found ${r.getClass}") + } + } + trace("Processor %d received request : %s".format(processor, requestDesc(true))) def updateRequestMetrics() { @@ -132,7 +143,7 @@ object RequestChannel extends Logging { val totalTime = endTimeMs - startTimeMs val fetchMetricNames = if (requestId == ApiKeys.FETCH.id) { - val isFromFollower = body.asInstanceOf[FetchRequest].isFromFollower + val isFromFollower = body[FetchRequest].isFromFollower Seq( if (isFromFollower) RequestMetrics.followFetchMetricName else RequestMetrics.consumerFetchMetricName @@ -163,11 +174,8 @@ object RequestChannel extends Logging { case class Response(processor: Int, request: Request, responseSend: Send, responseAction: ResponseAction) { request.responseCompleteTimeMs = Time.SYSTEM.milliseconds - def this(processor: Int, request: Request, responseSend: Send) = - this(processor, request, responseSend, if (responseSend == null) NoOpAction else SendAction) - - def this(request: Request, send: Send) = - this(request.processor, request, send) + def this(request: Request, responseSend: Send) = + this(request.processor, request, responseSend, if (responseSend == null) NoOpAction else SendAction) def this(request: Request, response: AbstractResponse) = this(request, response.toSend(request.connectionId, request.header)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 130821681ac6c..c90cace9bcfcb 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -108,7 +108,7 @@ class KafkaApis(val requestChannel: RequestChannel, request.requestObj.handleError(e, requestChannel, request) error("Error when handling request %s".format(request.requestObj), e) } else { - val response = request.body.getErrorResponse(e) + val response = request.body[AbstractRequest].getErrorResponse(e) /* If request doesn't have a default error response, we just close the connection. For example, when produce request has acks set to 0 */ @@ -128,7 +128,7 @@ class KafkaApis(val requestChannel: RequestChannel, // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val correlationId = request.header.correlationId - val leaderAndIsrRequest = request.body.asInstanceOf[LeaderAndIsrRequest] + val leaderAndIsrRequest = request.body[LeaderAndIsrRequest] try { def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]) { @@ -167,7 +167,7 @@ class KafkaApis(val requestChannel: RequestChannel, // ensureTopicExists is only for client facing requests // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted - val stopReplicaRequest = request.body.asInstanceOf[StopReplicaRequest] + val stopReplicaRequest = request.body[StopReplicaRequest] val response = if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { @@ -195,7 +195,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateMetadataRequest(request: RequestChannel.Request) { val correlationId = request.header.correlationId - val updateMetadataRequest = request.body.asInstanceOf[UpdateMetadataRequest] + val updateMetadataRequest = request.body[UpdateMetadataRequest] val updateMetadataResponse = if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { @@ -235,7 +235,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetCommitRequest(request: RequestChannel.Request) { val header = request.header - val offsetCommitRequest = request.body.asInstanceOf[OffsetCommitRequest] + val offsetCommitRequest = request.body[OffsetCommitRequest] // reject the request if not authorized to the group if (!authorize(request.session, Read, new Resource(Group, offsetCommitRequest.groupId))) { @@ -247,14 +247,13 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, response)) } else { val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = offsetCommitRequest.offsetData.asScala.toMap.partition { - case (topicPartition, _) => { + case (topicPartition, _) => val authorizedForDescribe = authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) val exists = metadataCache.contains(topicPartition.topic) if (!authorizedForDescribe && exists) debug(s"Offset commit request with correlation id ${header.correlationId} from client ${header.clientId} " + s"on partition $topicPartition failing due to user not having DESCRIBE authorization, but returning UNKNOWN_TOPIC_OR_PARTITION") authorizedForDescribe && exists - } } val (authorizedTopics, unauthorizedForReadTopics) = existingAndAuthorizedForDescribeTopics.partition { @@ -349,8 +348,8 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a produce request */ def handleProducerRequest(request: RequestChannel.Request) { - val produceRequest = request.body.asInstanceOf[ProduceRequest] - val numBytesAppended = request.header.sizeOf + produceRequest.sizeOf + val produceRequest = request.body[ProduceRequest] + val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = produceRequest.partitionRecords.asScala.partition { case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) && metadataCache.contains(topicPartition.topic) @@ -399,14 +398,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.noOperation(request.processor, request) } } else { - val respBody = request.header.apiVersion match { - case 0 => new ProduceResponse(mergedResponseStatus.asJava) - case version@(1 | 2) => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, version) - // This case shouldn't happen unless a new version of ProducerRequest is added without - // updating this part of the code to handle it properly. - case version => throw new IllegalArgumentException(s"Version `$version` of ProduceRequest is not handled. Code must be updated.") - } - + val respBody = new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, respBody)) } } @@ -445,7 +437,7 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a fetch request */ def handleFetchRequest(request: RequestChannel.Request) { - val fetchRequest = request.body.asInstanceOf[FetchRequest] + val fetchRequest = request.body[FetchRequest] val versionId = request.header.apiVersion val clientId = request.header.clientId @@ -505,13 +497,13 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.records.sizeInBytes) } - val response = new FetchResponse(versionId, fetchedPartitionData, 0) + val response = new FetchResponse(fetchedPartitionData, 0) + val responseStruct = response.toStruct(versionId) - def fetchResponseCallback(delayTimeMs: Int) { - trace(s"Sending fetch response to client $clientId of " + - s"${convertedPartitionData.map { case (_, v) => v.records.sizeInBytes }.sum} bytes") - val fetchResponse = if (delayTimeMs > 0) new FetchResponse(versionId, fetchedPartitionData, delayTimeMs) else response - requestChannel.sendResponse(new RequestChannel.Response(request, fetchResponse)) + def fetchResponseCallback(throttleTimeMs: Int) { + trace(s"Sending fetch response to client $clientId of ${responseStruct.sizeOf} bytes.") + val responseSend = response.toSend(responseStruct, throttleTimeMs, request.connectionId, request.header) + requestChannel.sendResponse(new RequestChannel.Response(request, responseSend)) } // When this callback is triggered, the remote API call has completed @@ -521,9 +513,10 @@ class KafkaApis(val requestChannel: RequestChannel, // We've already evaluated against the quota and are good to go. Just need to record it now. val responseSize = sizeOfThrottledPartitions(versionId, fetchRequest, mergedPartitionData, quotas.leader) quotas.leader.record(responseSize) - fetchResponseCallback(0) + fetchResponseCallback(throttleTimeMs = 0) } else { - quotas.fetch.recordAndMaybeThrottle(request.session.sanitizedUser, clientId, response.sizeOf, fetchResponseCallback) + quotas.fetch.recordAndMaybeThrottle(request.session.sanitizedUser, clientId, responseStruct.sizeOf, + fetchResponseCallback) } } @@ -547,7 +540,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest: FetchRequest, mergedPartitionData: Seq[(TopicPartition, FetchResponse.PartitionData)], quota: ReplicationQuotaManager): Int = { - val partitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]() + val partitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData] mergedPartitionData.foreach { case (tp, data) => if (quota.isThrottled(tp)) partitionData.put(tp, data) @@ -570,14 +563,14 @@ class KafkaApis(val requestChannel: RequestChannel, else handleListOffsetRequestV1(request) - val response = new ListOffsetResponse(mergedResponseMap.asJava, version) + val response = new ListOffsetResponse(mergedResponseMap.asJava) requestChannel.sendResponse(new RequestChannel.Response(request, response)) } private def handleListOffsetRequestV0(request : RequestChannel.Request) : Map[TopicPartition, ListOffsetResponse.PartitionData] = { val correlationId = request.header.correlationId val clientId = request.header.clientId - val offsetRequest = request.body.asInstanceOf[ListOffsetRequest] + val offsetRequest = request.body[ListOffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.offsetData.asScala.partition { case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) @@ -628,7 +621,7 @@ class KafkaApis(val requestChannel: RequestChannel, private def handleListOffsetRequestV1(request : RequestChannel.Request): Map[TopicPartition, ListOffsetResponse.PartitionData] = { val correlationId = request.header.correlationId val clientId = request.header.clientId - val offsetRequest = request.body.asInstanceOf[ListOffsetRequest] + val offsetRequest = request.body[ListOffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.partitionTimestamps.asScala.partition { case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) @@ -824,7 +817,7 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a topic metadata request */ def handleTopicMetadataRequest(request: RequestChannel.Request) { - val metadataRequest = request.body.asInstanceOf[MetadataRequest] + val metadataRequest = request.body[MetadataRequest] val requestVersion = request.header.apiVersion() val topics = @@ -889,8 +882,7 @@ class KafkaApis(val requestChannel: RequestChannel, brokers.map(_.getNode(request.listenerName)).asJava, clusterId, metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID), - completeTopicMetadata.asJava, - requestVersion + completeTopicMetadata.asJava ) requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) } @@ -900,7 +892,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val header = request.header - val offsetFetchRequest = request.body.asInstanceOf[OffsetFetchRequest] + val offsetFetchRequest = request.body[OffsetFetchRequest] def authorizeTopicDescribe(partition: TopicPartition) = authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) @@ -938,7 +930,7 @@ class KafkaApis(val requestChannel: RequestChannel, }.toMap val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap - new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava) } else { // versions 1 and above read offsets from Kafka if (offsetFetchRequest.isAllPartitions) { @@ -948,7 +940,7 @@ class KafkaApis(val requestChannel: RequestChannel, else { // clients are not allowed to see offsets for topics that are not authorized for Describe val authorizedPartitionData = allPartitionData.filter { case (topicPartition, _) => authorizeTopicDescribe(topicPartition) } - new OffsetFetchResponse(Errors.NONE, authorizedPartitionData.asJava, header.apiVersion) + new OffsetFetchResponse(Errors.NONE, authorizedPartitionData.asJava) } } else { val (authorizedPartitions, unauthorizedPartitions) = offsetFetchRequest.partitions.asScala @@ -959,7 +951,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest.getErrorResponse(error) else { val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap - new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava) } } } @@ -970,7 +962,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleGroupCoordinatorRequest(request: RequestChannel.Request) { - val groupCoordinatorRequest = request.body.asInstanceOf[GroupCoordinatorRequest] + val groupCoordinatorRequest = request.body[GroupCoordinatorRequest] if (!authorize(request.session, Describe, new Resource(Group, groupCoordinatorRequest.groupId))) { val responseBody = new GroupCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode) @@ -1003,7 +995,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeGroupRequest(request: RequestChannel.Request) { - val describeRequest = request.body.asInstanceOf[DescribeGroupsRequest] + val describeRequest = request.body[DescribeGroupsRequest] val groups = describeRequest.groupIds().asScala.map { groupId => if (!authorize(request.session, Describe, new Resource(Group, groupId))) { @@ -1036,12 +1028,12 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleJoinGroupRequest(request: RequestChannel.Request) { - val joinGroupRequest = request.body.asInstanceOf[JoinGroupRequest] + val joinGroupRequest = request.body[JoinGroupRequest] // the callback for sending a join-group response def sendResponseCallback(joinResult: JoinGroupResult) { val members = joinResult.members map { case (memberId, metadataArray) => (memberId, ByteBuffer.wrap(metadataArray)) } - val responseBody = new JoinGroupResponse(request.header.apiVersion, joinResult.error, joinResult.generationId, + val responseBody = new JoinGroupResponse(joinResult.error, joinResult.generationId, joinResult.subProtocol, joinResult.memberId, joinResult.leaderId, members.asJava) trace("Sending join group response %s for correlation id %d to client %s." @@ -1051,7 +1043,6 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authorize(request.session, Read, new Resource(Group, joinGroupRequest.groupId()))) { val responseBody = new JoinGroupResponse( - request.header.apiVersion, Errors.GROUP_AUTHORIZATION_FAILED, JoinGroupResponse.UNKNOWN_GENERATION_ID, JoinGroupResponse.UNKNOWN_PROTOCOL, @@ -1077,7 +1068,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleSyncGroupRequest(request: RequestChannel.Request) { - val syncGroupRequest = request.body.asInstanceOf[SyncGroupRequest] + val syncGroupRequest = request.body[SyncGroupRequest] def sendResponseCallback(memberState: Array[Byte], error: Errors) { val responseBody = new SyncGroupResponse(error, ByteBuffer.wrap(memberState)) @@ -1098,7 +1089,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleHeartbeatRequest(request: RequestChannel.Request) { - val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest] + val heartbeatRequest = request.body[HeartbeatRequest] // the callback for sending a heartbeat response def sendResponseCallback(error: Errors) { @@ -1123,7 +1114,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleLeaveGroupRequest(request: RequestChannel.Request) { - val leaveGroupRequest = request.body.asInstanceOf[LeaveGroupRequest] + val leaveGroupRequest = request.body[LeaveGroupRequest] // the callback for sending a leave-group response def sendResponseCallback(error: Errors) { @@ -1157,11 +1148,11 @@ class KafkaApis(val requestChannel: RequestChannel, // If this is considered to leak information about the broker version a workaround is to use SSL // with client authentication which is performed at an earlier stage of the connection where the // ApiVersionRequest is not available. - val responseBody = if (Protocol.apiVersionSupported(ApiKeys.API_VERSIONS.id, request.header.apiVersion)) - ApiVersionsResponse.API_VERSIONS_RESPONSE - else - ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION) - requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) + val responseSend = + if (Protocol.apiVersionSupported(ApiKeys.API_VERSIONS.id, request.header.apiVersion)) + ApiVersionsResponse.API_VERSIONS_RESPONSE.toSend(request.connectionId, request.header) + else ApiVersionsResponse.unsupportedVersionSend(request.connectionId, request.header) + requestChannel.sendResponse(new RequestChannel.Response(request, responseSend)) } def close() { @@ -1170,10 +1161,10 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreateTopicsRequest(request: RequestChannel.Request) { - val createTopicsRequest = request.body.asInstanceOf[CreateTopicsRequest] + val createTopicsRequest = request.body[CreateTopicsRequest] def sendResponseCallback(results: Map[String, CreateTopicsResponse.Error]): Unit = { - val responseBody = new CreateTopicsResponse(results.asJava, request.header.apiVersion) + val responseBody = new CreateTopicsResponse(results.asJava) trace(s"Sending create topics response $responseBody for correlation id ${request.header.correlationId} to client ${request.header.clientId}.") requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) } @@ -1220,7 +1211,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDeleteTopicsRequest(request: RequestChannel.Request) { - val deleteTopicRequest = request.body.asInstanceOf[DeleteTopicsRequest] + val deleteTopicRequest = request.body[DeleteTopicsRequest] val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = deleteTopicRequest.topics.asScala.partition { topic => authorize(request.session, Describe, new Resource(auth.Topic, topic)) && metadataCache.contains(topic) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c99d7c5fe2011..7fb02a30b549a 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -261,17 +261,13 @@ class ReplicaFetcherThread(name: String, } - private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long, consumerId: Int): Long = { + private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long, replicaId: Int): Long = { val requestBuilder = if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) { val partitions = Map(topicPartition -> (earliestOrLatest: java.lang.Long)) - new ListOffsetRequest.Builder(consumerId). - setTargetTimes(partitions.asJava). - setVersion(1) + ListOffsetRequest.Builder.forReplica(1, replicaId).setTargetTimes(partitions.asJava) } else { val partitions = Map(topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1)) - new ListOffsetRequest.Builder(consumerId). - setOffsetData(partitions.asJava). - setVersion(0) + ListOffsetRequest.Builder.forReplica(0, replicaId).setOffsetData(partitions.asJava) } val clientResponse = sendRequest(requestBuilder) val response = clientResponse.responseBody.asInstanceOf[ListOffsetResponse] @@ -295,9 +291,8 @@ class ReplicaFetcherThread(name: String, requestMap.put(topicPartition, new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize)) } - val requestBuilder = new JFetchRequest.Builder(maxWait, minBytes, requestMap). - setReplicaId(replicaId).setMaxBytes(maxBytes) - requestBuilder.setVersion(fetchRequestVersion) + val requestBuilder = JFetchRequest.Builder.forReplica(fetchRequestVersion, replicaId, maxWait, minBytes, requestMap) + .setMaxBytes(maxBytes) new FetchRequest(requestBuilder) } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 3285bf2ca560f..4f71258219d3f 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.errors._ -import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} import org.apache.kafka.common.requests._ import CreateTopicsRequest.TopicDetails import org.apache.kafka.common.security.auth.KafkaPrincipal @@ -191,11 +191,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def createFetchRequest = { val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData] partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 100)) - new requests.FetchRequest.Builder(100, Int.MaxValue, partitionMap).setReplicaId(5000).build() + val version = ProtoUtils.latestVersion(ApiKeys.FETCH.id) + requests.FetchRequest.Builder.forReplica(version, 5000, 100, Int.MaxValue, partitionMap).build() } private def createListOffsetsRequest = { - new requests.ListOffsetRequest.Builder().setTargetTimes( + requests.ListOffsetRequest.Builder.forConsumer(0).setTargetTimes( Map(tp -> (0L: java.lang.Long)).asJava). build() } @@ -214,7 +215,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId, Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava - new requests.UpdateMetadataRequest.Builder(brokerId, Int.MaxValue, partitionState, brokers).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, partitionState, brokers).build() } private def createJoinGroupRequest = { @@ -770,17 +772,18 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testUnauthorizedDeleteWithoutDescribe() { - val response = send(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val deleteResponse = DeleteTopicsResponse.parse(response) - + val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) + val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, deleteResponse.errors.asScala.head._2) } @Test def testUnauthorizedDeleteWithDescribe() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), deleteTopicResource) - val response = send(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val deleteResponse = DeleteTopicsResponse.parse(response) + val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) + val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, deleteResponse.errors.asScala.head._2) } @@ -788,8 +791,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testDeleteWithWildCardAuth() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Delete)), new Resource(Topic, "*")) - val response = send(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val deleteResponse = DeleteTopicsResponse.parse(response) + val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) + val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.NONE, deleteResponse.errors.asScala.head._2) } @@ -807,8 +811,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { isAuthorized: Boolean, isAuthorizedTopicDescribe: Boolean, topicExists: Boolean = true): AbstractResponse = { - val resp = send(request, apiKey) - val response = RequestKeyToResponseDeserializer(apiKey).getMethod("parse", classOf[ByteBuffer]).invoke(null, resp).asInstanceOf[AbstractResponse] + val resp = connectAndSend(request, apiKey) + val response = RequestKeyToResponseDeserializer(apiKey).getMethod("parse", classOf[ByteBuffer], classOf[Short]).invoke( + null, resp, request.version: java.lang.Short).asInstanceOf[AbstractResponse] val error = RequestKeyToError(apiKey).asInstanceOf[(AbstractResponse) => Errors](response) val authorizationErrorCodes = resources.flatMap { resourceType => @@ -877,7 +882,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def sendOffsetFetchRequest(request: requests.OffsetFetchRequest, socketServer: SocketServer): requests.OffsetFetchResponse = { - val response = send(request, ApiKeys.OFFSET_FETCH, socketServer) + val response = connectAndSend(request, ApiKeys.OFFSET_FETCH, socketServer) requests.OffsetFetchResponse.parse(response, request.version) } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 37bc23875fcfe..387560465af2b 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, NetworkSend} import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} import org.apache.kafka.common.record.MemoryRecords -import org.apache.kafka.common.requests.{ProduceRequest, RequestHeader} +import org.apache.kafka.common.requests.{AbstractRequest, ProduceRequest, RequestHeader} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Time import org.junit.Assert._ @@ -89,13 +89,11 @@ class SocketServerTest extends JUnitSuite { } def processRequest(channel: RequestChannel, request: RequestChannel.Request) { - val byteBuffer = ByteBuffer.allocate(request.header.sizeOf + request.body.sizeOf) - request.header.writeTo(byteBuffer) - request.body.writeTo(byteBuffer) + val byteBuffer = request.body[AbstractRequest].serialize(request.header) byteBuffer.rewind() val send = new NetworkSend(request.connectionId, byteBuffer) - channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) + channel.sendResponse(new RequestChannel.Response(request, send)) } def connect(s: SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { @@ -119,14 +117,11 @@ class SocketServerTest extends JUnitSuite { val ackTimeoutMs = 10000 val ack = 0: Short - val emptyRequest = new ProduceRequest.Builder( - ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = new ProduceRequest.Builder(ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() val emptyHeader = new RequestHeader(apiKey, emptyRequest.version, clientId, correlationId) - - val byteBuffer = ByteBuffer.allocate(emptyHeader.sizeOf + emptyRequest.sizeOf) - emptyHeader.writeTo(byteBuffer) - emptyRequest.writeTo(byteBuffer) + val byteBuffer = emptyRequest.serialize(emptyHeader) byteBuffer.rewind() + val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) serializedBytes @@ -289,13 +284,10 @@ class SocketServerTest extends JUnitSuite { val clientId = "" val ackTimeoutMs = 10000 val ack = 0: Short - val emptyRequest = new ProduceRequest.Builder( - ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = new ProduceRequest.Builder(ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() val emptyHeader = new RequestHeader(apiKey, emptyRequest.version, clientId, correlationId) - val byteBuffer = ByteBuffer.allocate(emptyHeader.sizeOf() + emptyRequest.sizeOf()) - emptyHeader.writeTo(byteBuffer) - emptyRequest.writeTo(byteBuffer) + val byteBuffer = emptyRequest.serialize(emptyHeader) byteBuffer.rewind() val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) @@ -355,7 +347,7 @@ class SocketServerTest extends JUnitSuite { // detected. If the buffer is larger than 102400 bytes, a second write is attempted and it fails with an // IOException. val send = new NetworkSend(request.connectionId, ByteBuffer.allocate(550000)) - channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) + channel.sendResponse(new RequestChannel.Response(request, send)) TestUtils.waitUntilTrue(() => totalTimeHistCount() == expectedTotalTimeCount, s"request metrics not updated, expected: $expectedTotalTimeCount, actual: ${totalTimeHistCount()}") diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index ef985311f271c..448fce1a54c2e 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -82,13 +82,13 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { protected def error(error: Errors, errorMessage: Option[String] = None): CreateTopicsResponse.Error = new CreateTopicsResponse.Error(error, errorMessage.orNull) - protected def duplicateFirstTopic(request: CreateTopicsRequest) = { + protected def toStructWithDuplicateFirstTopic(request: CreateTopicsRequest): Struct = { val struct = request.toStruct val topics = struct.getArray("create_topic_requests") val firstTopic = topics(0).asInstanceOf[Struct] val newTopics = firstTopic :: topics.toList struct.set("create_topic_requests", newTopics.toArray) - new CreateTopicsRequest(struct, request.version) + struct } protected def addPartitionsAndReplicationFactorToFirstTopic(request: CreateTopicsRequest) = { @@ -102,8 +102,10 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { protected def validateErrorCreateTopicsRequests(request: CreateTopicsRequest, expectedResponse: Map[String, CreateTopicsResponse.Error], - checkErrorMessage: Boolean = true): Unit = { - val response = sendCreateTopicRequest(request) + checkErrorMessage: Boolean = true, + requestStruct: Option[Struct] = None): Unit = { + val response = requestStruct.map(sendCreateTopicRequestStruct(_, request.version)).getOrElse( + sendCreateTopicRequest(request)) val errors = response.errors.asScala assertEquals("The response size should match", expectedResponse.size, response.errors.size) @@ -133,14 +135,20 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { assignments.map { case (k, v) => (k: Integer, v.map { i => i: Integer }.asJava) }.asJava } + protected def sendCreateTopicRequestStruct(requestStruct: Struct, apiVersion: Short, + socketServer: SocketServer = controllerSocketServer): CreateTopicsResponse = { + val response = connectAndSendStruct(requestStruct, ApiKeys.CREATE_TOPICS, apiVersion, socketServer) + CreateTopicsResponse.parse(response, apiVersion) + } + protected def sendCreateTopicRequest(request: CreateTopicsRequest, socketServer: SocketServer = controllerSocketServer): CreateTopicsResponse = { - val response = send(request, ApiKeys.CREATE_TOPICS, socketServer) + val response = connectAndSend(request, ApiKeys.CREATE_TOPICS, socketServer) CreateTopicsResponse.parse(response, request.version) } protected def sendMetadataRequest(request: MetadataRequest, destination: SocketServer = anySocketServer): MetadataResponse = { val version = ProtoUtils.latestVersion(ApiKeys.METADATA.id) - val response = send(request, ApiKeys.METADATA, destination = destination) + val response = connectAndSend(request, ApiKeys.METADATA, destination = destination) MetadataResponse.parse(response, version) } diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index ffe82d162e79d..248b91e9f211c 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -17,8 +17,7 @@ package kafka.server -import org.apache.kafka.common.protocol.types.Struct -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} import org.junit.Assert._ @@ -51,14 +50,13 @@ class ApiVersionsRequestTest extends BaseRequestTest { @Test def testApiVersionsRequestWithUnsupportedVersion() { - val apiVersionsRequest = new ApiVersionsRequest( - new Struct(ProtoUtils.currentRequestSchema(ApiKeys.API_VERSIONS.id)), Short.MaxValue) - val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest) + val apiVersionsRequest = new ApiVersionsRequest(0) + val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, Some(Short.MaxValue)) assertEquals(Errors.UNSUPPORTED_VERSION, apiVersionsResponse.error) } - private def sendApiVersionsRequest(request: ApiVersionsRequest): ApiVersionsResponse = { - val response = send(request, ApiKeys.API_VERSIONS) - ApiVersionsResponse.parse(response) + private def sendApiVersionsRequest(request: ApiVersionsRequest, apiVersion: Option[Short] = None): ApiVersionsResponse = { + val response = connectAndSend(request, ApiKeys.API_VERSIONS, apiVersion = apiVersion) + ApiVersionsResponse.parse(response, 0) } } diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index b864e5d0875fb..a26bc2e242413 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -26,8 +26,9 @@ import kafka.integration.KafkaServerTestHarness import kafka.network.SocketServer import kafka.utils._ import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.protocol.types.Struct import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} -import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader, ResponseHeader} +import org.apache.kafka.common.requests.{AbstractRequest, AbstractRequestResponse, RequestHeader, ResponseHeader} abstract class BaseRequestTest extends KafkaServerTestHarness { private var correlationId = 0 @@ -97,41 +98,64 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { } /** - * - * @param request - * @param apiKey * @param destination An optional SocketServer ot send the request to. If not set, any available server is used. * @param protocol An optional SecurityProtocol to use. If not set, PLAINTEXT is used. - * @return + * @return A ByteBuffer containing the response (without the response header) */ - def send(request: AbstractRequest, apiKey: ApiKeys, - destination: SocketServer = anySocketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): ByteBuffer = { + def connectAndSend(request: AbstractRequest, apiKey: ApiKeys, + destination: SocketServer = anySocketServer, + apiVersion: Option[Short] = None, + protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): ByteBuffer = { val socket = connect(destination, protocol) - try { - send(request, apiKey, socket) - } finally { - socket.close() - } + try send(request, apiKey, socket, apiVersion) + finally socket.close() } /** - * Serializes and send the request to the given api. + * @param destination An optional SocketServer ot send the request to. If not set, any available server is used. + * @param protocol An optional SecurityProtocol to use. If not set, PLAINTEXT is used. + * @return A ByteBuffer containing the response (without the response header). + */ + def connectAndSendStruct(requestStruct: Struct, apiKey: ApiKeys, apiVersion: Short, + destination: SocketServer = anySocketServer, + protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): ByteBuffer = { + val socket = connect(destination, protocol) + try sendStruct(requestStruct, apiKey, socket, apiVersion) + finally socket.close() + } + + /** + * Serializes and sends the request to the given api. * A ByteBuffer containing the response is returned. */ - def send(request: AbstractRequest, apiKey: ApiKeys, socket: Socket): ByteBuffer = { - correlationId += 1 - val serializedBytes = { - val header = new RequestHeader(apiKey.id, request.version, "client-id", correlationId) - val byteBuffer = ByteBuffer.allocate(header.sizeOf() + request.sizeOf) - header.writeTo(byteBuffer) - request.writeTo(byteBuffer) - byteBuffer.array() - } + def send(request: AbstractRequest, apiKey: ApiKeys, socket: Socket, apiVersion: Option[Short] = None): ByteBuffer = { + val header = nextRequestHeader(apiKey, apiVersion.getOrElse(request.version)) + val serializedBytes = request.serialize(header).array + val response = requestAndReceive(socket, serializedBytes) + skipResponseHeader(response) + } + /** + * Serializes and sends the requestStruct to the given api. + * A ByteBuffer containing the response (without the response header) is returned. + */ + def sendStruct(requestStruct: Struct, apiKey: ApiKeys, socket: Socket, apiVersion: Short): ByteBuffer = { + val header = nextRequestHeader(apiKey, apiVersion) + val serializedBytes = AbstractRequestResponse.serialize(header.toStruct, requestStruct).array val response = requestAndReceive(socket, serializedBytes) + skipResponseHeader(response) + } + private def skipResponseHeader(response: Array[Byte]): ByteBuffer = { val responseBuffer = ByteBuffer.wrap(response) - ResponseHeader.parse(responseBuffer) // Parse the header to ensure its valid and move the buffer forward + // Parse the header to ensure its valid and move the buffer forward + ResponseHeader.parse(responseBuffer) responseBuffer } + + def nextRequestHeader(apiKey: ApiKeys, apiVersion: Short): RequestHeader = { + correlationId += 1 + new RequestHeader(apiKey.id, apiVersion, "client-id", correlationId) + } + } diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index 6efa189a6c828..4ab952032a4cf 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -116,30 +116,27 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { // Duplicate val singleRequest = new CreateTopicsRequest.Builder(Map("duplicate-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, 1000).build() - val duplicateRequest = duplicateFirstTopic(singleRequest) - assertFalse("Request doesn't have duplicate topics", duplicateRequest.duplicateTopics().isEmpty) - validateErrorCreateTopicsRequests(duplicateRequest, Map("duplicate-topic" -> error(Errors.INVALID_REQUEST, - Some("""Create topics request from client `client-id` contains multiple entries for the following topics: duplicate-topic""")))) + validateErrorCreateTopicsRequests(singleRequest, Map("duplicate-topic" -> error(Errors.INVALID_REQUEST, + Some("""Create topics request from client `client-id` contains multiple entries for the following topics: duplicate-topic"""))), + requestStruct = Some(toStructWithDuplicateFirstTopic(singleRequest))) // Duplicate Partial with validateOnly val doubleRequestValidateOnly = new CreateTopicsRequest.Builder(Map( "duplicate-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort), "other-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, 1000, true).build() - val duplicateDoubleRequestValidateOnly = duplicateFirstTopic(doubleRequestValidateOnly) - assertFalse("Request doesn't have duplicate topics", duplicateDoubleRequestValidateOnly.duplicateTopics.isEmpty) - validateErrorCreateTopicsRequests(duplicateDoubleRequestValidateOnly, Map( + validateErrorCreateTopicsRequests(doubleRequestValidateOnly, Map( "duplicate-topic" -> error(Errors.INVALID_REQUEST), - "other-topic" -> error(Errors.NONE)), checkErrorMessage = false) + "other-topic" -> error(Errors.NONE)), checkErrorMessage = false, + requestStruct = Some(toStructWithDuplicateFirstTopic(doubleRequestValidateOnly))) // Duplicate Partial val doubleRequest = new CreateTopicsRequest.Builder(Map( "duplicate-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort), "other-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, 1000).build() - val duplicateDoubleRequest = duplicateFirstTopic(doubleRequest) - assertFalse("Request doesn't have duplicate topics", duplicateDoubleRequest.duplicateTopics.isEmpty) - validateErrorCreateTopicsRequests(duplicateDoubleRequest, Map( + validateErrorCreateTopicsRequests(doubleRequest, Map( "duplicate-topic" -> error(Errors.INVALID_REQUEST), - "other-topic" -> error(Errors.NONE)), checkErrorMessage = false) + "other-topic" -> error(Errors.NONE)), checkErrorMessage = false, + requestStruct = Some(toStructWithDuplicateFirstTopic(doubleRequest))) // Partitions/ReplicationFactor and ReplicaAssignment val assignments = replicaAssignmentToJava(Map(0 -> List(0))) diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala index 9a092d046b48e..9cd53d8e15888 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala @@ -112,12 +112,12 @@ class DeleteTopicsRequestTest extends BaseRequestTest { } private def sendDeleteTopicsRequest(request: DeleteTopicsRequest, socketServer: SocketServer = controllerSocketServer): DeleteTopicsResponse = { - val response = send(request, ApiKeys.DELETE_TOPICS, socketServer) + val response = connectAndSend(request, ApiKeys.DELETE_TOPICS, socketServer) DeleteTopicsResponse.parse(response, request.version) } private def sendMetadataRequest(request: MetadataRequest): MetadataResponse = { - val response = send(request, ApiKeys.METADATA) - MetadataResponse.parse(response) + val response = connectAndSend(request, ApiKeys.METADATA) + MetadataResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 1f9e18ba805fd..2d4a22a473338 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -114,15 +114,15 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val correlationId = -1 TestUtils.createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = servers) + val version = 2: Short val serializedBytes = { - val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, 2, null, correlationId) + val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, null, correlationId) val messageBytes = "message".getBytes val records = MemoryRecords.readableRecords(ByteBuffer.wrap(messageBytes)) - val request = new ProduceRequest.Builder( - 1, 10000, Map(topicPartition -> records).asJava).build() - val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.sizeOf) + val request = new ProduceRequest.Builder(1, 10000, Map(topicPartition -> records).asJava).build() + val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.toStruct.sizeOf) byteBuffer.put(headerBytes) - request.writeTo(byteBuffer) + request.toStruct.writeTo(byteBuffer) byteBuffer.array() } @@ -130,13 +130,13 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val responseBuffer = ByteBuffer.wrap(response) val responseHeader = ResponseHeader.parse(responseBuffer) - val produceResponse = ProduceResponse.parse(responseBuffer) + val produceResponse = ProduceResponse.parse(responseBuffer, version) - assertEquals("The response should parse completely", 0, responseBuffer.remaining()) - assertEquals("The correlationId should match request", correlationId, responseHeader.correlationId()) - assertEquals("One partition response should be returned", 1, produceResponse.responses().size()) + assertEquals("The response should parse completely", 0, responseBuffer.remaining) + assertEquals("The correlationId should match request", correlationId, responseHeader.correlationId) + assertEquals("One partition response should be returned", 1, produceResponse.responses.size) - val partitionResponse = produceResponse.responses().get(topicPartition) + val partitionResponse = produceResponse.responses.get(topicPartition) assertNotNull(partitionResponse) assertEquals("There should be no error", Errors.NONE, partitionResponse.error) } diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 38113609434f8..64be5b3088893 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -24,7 +24,7 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} import org.apache.kafka.common.record.LogEntry import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer @@ -56,7 +56,7 @@ class FetchRequestTest extends BaseRequestTest { private def createFetchRequest(maxResponseBytes: Int, maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = - new FetchRequest.Builder(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) + FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) .setMaxBytes(maxResponseBytes).build() private def createPartitionMap(maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], @@ -69,8 +69,8 @@ class FetchRequestTest extends BaseRequestTest { } private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = { - val response = send(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) - FetchResponse.parse(response) + val response = connectAndSend(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) + FetchResponse.parse(response, ProtoUtils.latestVersion(ApiKeys.FETCH.id)) } @Test @@ -156,10 +156,9 @@ class FetchRequestTest extends BaseRequestTest { val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head producer.send(new ProducerRecord(topicPartition.topic, topicPartition.partition, "key", new String(new Array[Byte](maxPartitionBytes + 1)))).get - val fetchRequestBuilder = new FetchRequest.Builder( - Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, Seq(topicPartition))). - setVersion(2) - val fetchResponse = sendFetchRequest(leaderId, fetchRequestBuilder.build()) + val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, + createPartitionMap(maxPartitionBytes, Seq(topicPartition))).build(2) + val fetchResponse = sendFetchRequest(leaderId, fetchRequest) val partitionData = fetchResponse.responseData.get(topicPartition) assertEquals(Errors.NONE, partitionData.error) assertTrue(partitionData.highWatermark > 0) diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 99a95adb9e455..1b5007dbd41f7 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -22,7 +22,7 @@ import util.Arrays.asList import kafka.common.BrokerEndPointNotAvailableException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} import org.apache.kafka.common.requests.{PartitionState, UpdateMetadataRequest} import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} import org.junit.Test @@ -69,8 +69,9 @@ class MetadataCacheTest { new TopicPartition(topic, 1) -> new PartitionState(controllerEpoch, 1, 1, asList(1), zkVersion, asSet(1)), new TopicPartition(topic, 2) -> new PartitionState(controllerEpoch, 2, 2, asList(2), zkVersion, asSet(2))) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) for (securityProtocol <- Seq(SecurityProtocol.PLAINTEXT, SecurityProtocol.SSL)) { @@ -120,8 +121,9 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asSet(0))) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName) @@ -163,8 +165,9 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) // Validate errorUnavailableEndpoints = false @@ -222,8 +225,9 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) // Validate errorUnavailableEndpoints = false @@ -273,8 +277,9 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, + brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) try { @@ -305,8 +310,9 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, + brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) } diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index f3bb912e0260b..ed0e8051ffd0b 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -36,14 +36,14 @@ class MetadataRequestTest extends BaseRequestTest { @Test def testClusterIdWithRequestVersion1() { - val v1MetadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) val v1ClusterId = v1MetadataResponse.clusterId assertNull(s"v1 clusterId should be null", v1ClusterId) } @Test def testClusterIdIsValid() { - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(2.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(2.toShort)) isValidClusterId(metadataResponse.clusterId) } @@ -51,7 +51,7 @@ class MetadataRequestTest extends BaseRequestTest { def testControllerId() { val controllerServer = servers.find(_.kafkaController.isActive).get val controllerId = controllerServer.config.brokerId - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) assertEquals("Controller id should match the active controller", controllerId, metadataResponse.controller.id) @@ -64,14 +64,14 @@ class MetadataRequestTest extends BaseRequestTest { val controllerId2 = controllerServer2.config.brokerId assertNotEquals("Controller id should switch to a new broker", controllerId, controllerId2) TestUtils.waitUntilTrue(() => { - val metadataResponse2 = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse2 = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) metadataResponse2.controller != null && controllerServer2.apis.brokerId == metadataResponse2.controller.id }, "Controller id should match the active controller after failover", 5000) } @Test def testRack() { - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) // Validate rack matches what's set in generateConfigs() above metadataResponse.brokers.asScala.foreach { broker => assertEquals("Rack information should match config", s"rack/${broker.id}", broker.rack) @@ -86,7 +86,7 @@ class MetadataRequestTest extends BaseRequestTest { TestUtils.createTopic(zkUtils, internalTopic, 3, 2, servers) TestUtils.createTopic(zkUtils, notInternalTopic, 3, 2, servers) - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) assertTrue("Response should have no errors", metadataResponse.errors.isEmpty) val topicMetadata = metadataResponse.topicMetadata.asScala @@ -124,7 +124,7 @@ class MetadataRequestTest extends BaseRequestTest { assertEquals("V0 Response should have 2 (all) topics", 2, metadataResponseV0.topicMetadata.size()) // v1, Null represents all topics - val metadataResponseV1 = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponseV1 = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) assertTrue("V1 Response should have no errors", metadataResponseV1.errors.isEmpty) assertEquals("V1 Response should have 2 (all) topics", 2, metadataResponseV1.topicMetadata.size()) } @@ -177,7 +177,7 @@ class MetadataRequestTest extends BaseRequestTest { } private def sendMetadataRequest(request: MetadataRequest): MetadataResponse = { - val response = send(request, ApiKeys.METADATA) + val response = connectAndSend(request, ApiKeys.METADATA) MetadataResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index b05be9d580531..81118fa84b04e 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -88,8 +88,8 @@ class ProduceRequestTest extends BaseRequestTest { } private def sendProduceRequest(leaderId: Int, request: ProduceRequest): ProduceResponse = { - val response = send(request, ApiKeys.PRODUCE, destination = brokerSocketServer(leaderId)) - ProduceResponse.parse(response) + val response = connectAndSend(request, ApiKeys.PRODUCE, destination = brokerSocketServer(leaderId)) + ProduceResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 927ace9b42b45..92a518dc852c5 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -20,14 +20,13 @@ import java.io.IOException import java.net.Socket import java.util.Collections -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} import org.apache.kafka.common.requests.SaslHandshakeRequest import org.apache.kafka.common.requests.SaslHandshakeResponse import org.junit.Test import org.junit.Assert._ import kafka.api.SaslTestHarness -import org.apache.kafka.common.protocol.types.Struct class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT @@ -42,8 +41,7 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { def testApiVersionsRequestBeforeSaslHandshakeRequest() { val plaintextSocket = connect(protocol = securityProtocol) try { - val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, - new ApiVersionsRequest.Builder().setVersion(0).build()) + val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest.Builder().build(0)) ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse) sendSaslHandshakeRequestValidateResponse(plaintextSocket) } finally { @@ -57,8 +55,7 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { try { sendSaslHandshakeRequestValidateResponse(plaintextSocket) try { - sendApiVersionsRequest(plaintextSocket, - new ApiVersionsRequest.Builder().setVersion(0).build()) + sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest.Builder().build(0)) fail("Versions Request during Sasl handshake did not fail") } catch { case _: IOException => // expected exception @@ -72,12 +69,10 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { def testApiVersionsRequestWithUnsupportedVersion() { val plaintextSocket = connect(protocol = securityProtocol) try { - val apiVersionsRequest = new ApiVersionsRequest( - new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, 0)), Short.MaxValue); - val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, apiVersionsRequest) + val apiVersionsRequest = new ApiVersionsRequest(0) + val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, apiVersionsRequest, Some(Short.MaxValue)) assertEquals(Errors.UNSUPPORTED_VERSION, apiVersionsResponse.error) - val apiVersionsResponse2 = sendApiVersionsRequest(plaintextSocket, - new ApiVersionsRequest.Builder().setVersion(0).build()) + val apiVersionsResponse2 = sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest.Builder().build(0)) ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse2) sendSaslHandshakeRequestValidateResponse(plaintextSocket) } finally { @@ -85,15 +80,17 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { } } - private def sendApiVersionsRequest(socket: Socket, request: ApiVersionsRequest): ApiVersionsResponse = { - val response = send(request, ApiKeys.API_VERSIONS, socket) - ApiVersionsResponse.parse(response) + private def sendApiVersionsRequest(socket: Socket, request: ApiVersionsRequest, + apiVersion: Option[Short] = None): ApiVersionsResponse = { + val response = send(request, ApiKeys.API_VERSIONS, socket, apiVersion) + ApiVersionsResponse.parse(response, request.version) } private def sendSaslHandshakeRequestValidateResponse(socket: Socket) { - val response = send(new SaslHandshakeRequest("PLAIN"), ApiKeys.SASL_HANDSHAKE, socket) - val handshakeResponse = SaslHandshakeResponse.parse(response) + val request = new SaslHandshakeRequest("PLAIN") + val response = send(request, ApiKeys.SASL_HANDSHAKE, socket) + val handshakeResponse = SaslHandshakeResponse.parse(response, request.version) assertEquals(Errors.NONE, handshakeResponse.error) - assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms()) + assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms) } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index 69b3c46d162bf..539965390e431 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -106,8 +106,8 @@ public MetadataResponse fetchMetadata() { MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList(), new ArrayList()); MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE, topic, true, Collections.singletonList(partitionMetadata)); MetadataResponse response = new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, - Collections.singletonList(topicMetadata), 0); + Collections.singletonList(topicMetadata)); return response; } } -} \ No newline at end of file +}