KAFKA-10818: Skip conversion to `Struct` when serializing generated requests/responses (#7409)

Generated request/response classes have code to serialize/deserialize directly to
`ByteBuffer` so the intermediate conversion to `Struct` can be skipped for them.
We have recently completed the transition to generated request/response classes,
so we can also remove the `Struct` based fallbacks.

Additional noteworthy changes:
* `AbstractRequest.parseRequest` has a more efficient computation of request size that
relies on the received buffer instead of the parsed `Struct`.
* Use `SendBuilder` for `AbstractRequest/Response` `toSend`, made the superclass
implementation final and removed the overrides that are no longer necessary.
* Removed request/response constructors that assume latest version as they are unsafe
outside of tests.
* Removed redundant version fields in requests/responses.
* Removed unnecessary work in `OffsetFetchResponse`'s constructor when version >= 2.
* Made `AbstractResponse.throttleTimeMs()` abstract.
* Using `toSend` in `SaslClientAuthenticator` instead of `serialize`.
* Various changes in Request/Response classes to make them more consistent and to
rely on the Data classes as much as possible when it comes to their state.
* Remove the version argument from `AbstractResponse.toString`.
* Fix `getErrorResponse` for `ProduceRequest` and `DescribeClientQuotasRequest` to
use `ApiError` which processes the error message sent back to the clients. This was
uncovered by an accidental fix to a `RequestResponseTest` test (it was calling
`AbstractResponse.toString` instead of `AbstractResponse.toString(short)`).

Rely on existing protocol tests to ensure this refactoring does not change 
observed behavior (aside from improved performance).

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Ismael Juma 2020-12-07 15:39:57 -08:00 committed by GitHub
parent ab0807dd85
commit 6f27bb02da
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
191 changed files with 1650 additions and 2509 deletions

View File

@ -83,14 +83,14 @@ public final class ClientRequest {
} }
public RequestHeader makeHeader(short version) { public RequestHeader makeHeader(short version) {
short requestApiKey = requestBuilder.apiKey().id; ApiKeys requestApiKey = apiKey();
return new RequestHeader( return new RequestHeader(
new RequestHeaderData() new RequestHeaderData()
.setRequestApiKey(requestApiKey) .setRequestApiKey(requestApiKey.id)
.setRequestApiVersion(version) .setRequestApiVersion(version)
.setClientId(clientId) .setClientId(clientId)
.setCorrelationId(correlationId), .setCorrelationId(correlationId),
ApiKeys.forId(requestApiKey).requestHeaderVersion(version)); requestApiKey.requestHeaderVersion(version));
} }
public AbstractRequest.Builder<?> requestBuilder() { public AbstractRequest.Builder<?> requestBuilder() {

View File

@ -870,9 +870,8 @@ public class NetworkClient implements KafkaClient {
InFlightRequest req = inFlightRequests.completeNext(source); InFlightRequest req = inFlightRequests.completeNext(source);
AbstractResponse response = parseResponse(receive.payload(), req.header); AbstractResponse response = parseResponse(receive.payload(), req.header);
if (throttleTimeSensor != null) { if (throttleTimeSensor != null)
throttleTimeSensor.record(response.throttleTimeMs()); throttleTimeSensor.record(response.throttleTimeMs(), now);
}
if (log.isDebugEnabled()) { if (log.isDebugEnabled()) {
log.debug("Received {} response from node {} for request with header {}: {}", log.debug("Received {} response from node {} for request with header {}: {}",

View File

@ -1182,8 +1182,7 @@ public class KafkaAdminClient extends AdminClient {
try { try {
call.handleResponse(response.responseBody()); call.handleResponse(response.responseBody());
if (log.isTraceEnabled()) if (log.isTraceEnabled())
log.trace("{} got response {}", call, log.trace("{} got response {}", call, response.responseBody());
response.responseBody().toString(response.requestHeader().apiVersion()));
} catch (Throwable t) { } catch (Throwable t) {
if (log.isTraceEnabled()) if (log.isTraceEnabled())
log.trace("{} handleResponse failed with {}", call, prettyPrintException(t)); log.trace("{} handleResponse failed with {}", call, prettyPrintException(t));

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.record.BaseRecords;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MultiRecordsSend; import org.apache.kafka.common.record.MultiRecordsSend;
import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.requests.RequestUtils;
import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.requests.ResponseHeader;
import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.ByteUtils;
@ -181,7 +182,7 @@ public class SendBuilder implements Writable {
public static Send buildRequestSend( public static Send buildRequestSend(
String destination, String destination,
RequestHeader header, RequestHeader header,
ApiMessage apiRequest Message apiRequest
) { ) {
return buildSend( return buildSend(
destination, destination,
@ -195,7 +196,7 @@ public class SendBuilder implements Writable {
public static Send buildResponseSend( public static Send buildResponseSend(
String destination, String destination,
ResponseHeader header, ResponseHeader header,
ApiMessage apiResponse, Message apiResponse,
short apiVersion short apiVersion
) { ) {
return buildSend( return buildSend(
@ -209,16 +210,13 @@ public class SendBuilder implements Writable {
private static Send buildSend( private static Send buildSend(
String destination, String destination,
ApiMessage header, Message header,
short headerVersion, short headerVersion,
ApiMessage apiMessage, Message apiMessage,
short apiVersion short apiVersion
) { ) {
ObjectSerializationCache serializationCache = new ObjectSerializationCache(); ObjectSerializationCache serializationCache = new ObjectSerializationCache();
MessageSizeAccumulator messageSize = new MessageSizeAccumulator(); MessageSizeAccumulator messageSize = RequestUtils.size(serializationCache, header, headerVersion, apiMessage, apiVersion);
header.addSize(messageSize, serializationCache, headerVersion);
apiMessage.addSize(messageSize, serializationCache, apiVersion);
int totalSize = messageSize.totalSize(); int totalSize = messageSize.totalSize();
int sizeExcludingZeroCopyFields = totalSize - messageSize.zeroCopySize(); int sizeExcludingZeroCopyFields = totalSize - messageSize.zeroCopySize();

View File

@ -17,14 +17,12 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.AlterIsrRequestData;
import org.apache.kafka.common.message.ProduceRequestData;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.protocol.SendBuilder;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -79,13 +77,13 @@ public abstract class AbstractRequest implements AbstractRequestResponse {
} }
private final short version; private final short version;
public final ApiKeys api; private final ApiKeys apiKey;
public AbstractRequest(ApiKeys api, short version) { public AbstractRequest(ApiKeys apiKey, short version) {
if (!api.isVersionSupported(version)) if (!apiKey.isVersionSupported(version))
throw new UnsupportedVersionException("The " + api + " protocol does not support version " + version); throw new UnsupportedVersionException("The " + apiKey + " protocol does not support version " + version);
this.version = version; this.version = version;
this.api = api; this.apiKey = apiKey;
} }
/** /**
@ -95,21 +93,33 @@ public abstract class AbstractRequest implements AbstractRequestResponse {
return version; return version;
} }
public Send toSend(String destination, RequestHeader header) { public ApiKeys apiKey() {
return new NetworkSend(destination, serialize(header)); return apiKey;
} }
/** public final Send toSend(String destination, RequestHeader header) {
* Use with care, typically {@link #toSend(String, RequestHeader)} should be used instead. return SendBuilder.buildRequestSend(destination, header, data());
*/
public ByteBuffer serialize(RequestHeader header) {
return RequestUtils.serialize(header.toStruct(), toStruct());
} }
protected abstract Struct toStruct(); // Visible for testing
public final ByteBuffer serializeWithHeader(RequestHeader header) {
return RequestUtils.serialize(header.data(), header.headerVersion(), data(), version);
}
protected abstract Message data();
// Visible for testing
public final ByteBuffer serializeBody() {
return RequestUtils.serialize(null, (short) 0, data(), version);
}
// Visible for testing
final int sizeInBytes() {
return data().size(new ObjectSerializationCache(), version);
}
public String toString(boolean verbose) { public String toString(boolean verbose) {
return toStruct().toString(); return data().toString();
} }
@Override @Override
@ -144,126 +154,131 @@ public abstract class AbstractRequest implements AbstractRequestResponse {
/** /**
* Factory method for getting a request object based on ApiKey ID and a version * Factory method for getting a request object based on ApiKey ID and a version
*/ */
public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Struct struct) { public static RequestAndSize parseRequest(ApiKeys apiKey, short apiVersion, ByteBuffer buffer) {
int bufferSize = buffer.remaining();
return new RequestAndSize(doParseRequest(apiKey, apiVersion, buffer), bufferSize);
}
private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, ByteBuffer buffer) {
switch (apiKey) { switch (apiKey) {
case PRODUCE: case PRODUCE:
return new ProduceRequest(new ProduceRequestData(struct, apiVersion), apiVersion); return ProduceRequest.parse(buffer, apiVersion);
case FETCH: case FETCH:
return new FetchRequest(new FetchRequestData(struct, apiVersion), apiVersion); return FetchRequest.parse(buffer, apiVersion);
case LIST_OFFSETS: case LIST_OFFSETS:
return new ListOffsetRequest(struct, apiVersion); return ListOffsetRequest.parse(buffer, apiVersion);
case METADATA: case METADATA:
return new MetadataRequest(struct, apiVersion); return MetadataRequest.parse(buffer, apiVersion);
case OFFSET_COMMIT: case OFFSET_COMMIT:
return new OffsetCommitRequest(struct, apiVersion); return OffsetCommitRequest.parse(buffer, apiVersion);
case OFFSET_FETCH: case OFFSET_FETCH:
return new OffsetFetchRequest(struct, apiVersion); return OffsetFetchRequest.parse(buffer, apiVersion);
case FIND_COORDINATOR: case FIND_COORDINATOR:
return new FindCoordinatorRequest(struct, apiVersion); return FindCoordinatorRequest.parse(buffer, apiVersion);
case JOIN_GROUP: case JOIN_GROUP:
return new JoinGroupRequest(struct, apiVersion); return JoinGroupRequest.parse(buffer, apiVersion);
case HEARTBEAT: case HEARTBEAT:
return new HeartbeatRequest(struct, apiVersion); return HeartbeatRequest.parse(buffer, apiVersion);
case LEAVE_GROUP: case LEAVE_GROUP:
return new LeaveGroupRequest(struct, apiVersion); return LeaveGroupRequest.parse(buffer, apiVersion);
case SYNC_GROUP: case SYNC_GROUP:
return new SyncGroupRequest(struct, apiVersion); return SyncGroupRequest.parse(buffer, apiVersion);
case STOP_REPLICA: case STOP_REPLICA:
return new StopReplicaRequest(struct, apiVersion); return StopReplicaRequest.parse(buffer, apiVersion);
case CONTROLLED_SHUTDOWN: case CONTROLLED_SHUTDOWN:
return new ControlledShutdownRequest(struct, apiVersion); return ControlledShutdownRequest.parse(buffer, apiVersion);
case UPDATE_METADATA: case UPDATE_METADATA:
return new UpdateMetadataRequest(struct, apiVersion); return UpdateMetadataRequest.parse(buffer, apiVersion);
case LEADER_AND_ISR: case LEADER_AND_ISR:
return new LeaderAndIsrRequest(struct, apiVersion); return LeaderAndIsrRequest.parse(buffer, apiVersion);
case DESCRIBE_GROUPS: case DESCRIBE_GROUPS:
return new DescribeGroupsRequest(struct, apiVersion); return DescribeGroupsRequest.parse(buffer, apiVersion);
case LIST_GROUPS: case LIST_GROUPS:
return new ListGroupsRequest(struct, apiVersion); return ListGroupsRequest.parse(buffer, apiVersion);
case SASL_HANDSHAKE: case SASL_HANDSHAKE:
return new SaslHandshakeRequest(struct, apiVersion); return SaslHandshakeRequest.parse(buffer, apiVersion);
case API_VERSIONS: case API_VERSIONS:
return new ApiVersionsRequest(struct, apiVersion); return ApiVersionsRequest.parse(buffer, apiVersion);
case CREATE_TOPICS: case CREATE_TOPICS:
return new CreateTopicsRequest(struct, apiVersion); return CreateTopicsRequest.parse(buffer, apiVersion);
case DELETE_TOPICS: case DELETE_TOPICS:
return new DeleteTopicsRequest(struct, apiVersion); return DeleteTopicsRequest.parse(buffer, apiVersion);
case DELETE_RECORDS: case DELETE_RECORDS:
return new DeleteRecordsRequest(struct, apiVersion); return DeleteRecordsRequest.parse(buffer, apiVersion);
case INIT_PRODUCER_ID: case INIT_PRODUCER_ID:
return new InitProducerIdRequest(struct, apiVersion); return InitProducerIdRequest.parse(buffer, apiVersion);
case OFFSET_FOR_LEADER_EPOCH: case OFFSET_FOR_LEADER_EPOCH:
return new OffsetsForLeaderEpochRequest(struct, apiVersion); return OffsetsForLeaderEpochRequest.parse(buffer, apiVersion);
case ADD_PARTITIONS_TO_TXN: case ADD_PARTITIONS_TO_TXN:
return new AddPartitionsToTxnRequest(struct, apiVersion); return AddPartitionsToTxnRequest.parse(buffer, apiVersion);
case ADD_OFFSETS_TO_TXN: case ADD_OFFSETS_TO_TXN:
return new AddOffsetsToTxnRequest(struct, apiVersion); return AddOffsetsToTxnRequest.parse(buffer, apiVersion);
case END_TXN: case END_TXN:
return new EndTxnRequest(struct, apiVersion); return EndTxnRequest.parse(buffer, apiVersion);
case WRITE_TXN_MARKERS: case WRITE_TXN_MARKERS:
return new WriteTxnMarkersRequest(struct, apiVersion); return WriteTxnMarkersRequest.parse(buffer, apiVersion);
case TXN_OFFSET_COMMIT: case TXN_OFFSET_COMMIT:
return new TxnOffsetCommitRequest(struct, apiVersion); return TxnOffsetCommitRequest.parse(buffer, apiVersion);
case DESCRIBE_ACLS: case DESCRIBE_ACLS:
return new DescribeAclsRequest(struct, apiVersion); return DescribeAclsRequest.parse(buffer, apiVersion);
case CREATE_ACLS: case CREATE_ACLS:
return new CreateAclsRequest(struct, apiVersion); return CreateAclsRequest.parse(buffer, apiVersion);
case DELETE_ACLS: case DELETE_ACLS:
return new DeleteAclsRequest(struct, apiVersion); return DeleteAclsRequest.parse(buffer, apiVersion);
case DESCRIBE_CONFIGS: case DESCRIBE_CONFIGS:
return new DescribeConfigsRequest(struct, apiVersion); return DescribeConfigsRequest.parse(buffer, apiVersion);
case ALTER_CONFIGS: case ALTER_CONFIGS:
return new AlterConfigsRequest(struct, apiVersion); return AlterConfigsRequest.parse(buffer, apiVersion);
case ALTER_REPLICA_LOG_DIRS: case ALTER_REPLICA_LOG_DIRS:
return new AlterReplicaLogDirsRequest(struct, apiVersion); return AlterReplicaLogDirsRequest.parse(buffer, apiVersion);
case DESCRIBE_LOG_DIRS: case DESCRIBE_LOG_DIRS:
return new DescribeLogDirsRequest(struct, apiVersion); return DescribeLogDirsRequest.parse(buffer, apiVersion);
case SASL_AUTHENTICATE: case SASL_AUTHENTICATE:
return new SaslAuthenticateRequest(struct, apiVersion); return SaslAuthenticateRequest.parse(buffer, apiVersion);
case CREATE_PARTITIONS: case CREATE_PARTITIONS:
return new CreatePartitionsRequest(struct, apiVersion); return CreatePartitionsRequest.parse(buffer, apiVersion);
case CREATE_DELEGATION_TOKEN: case CREATE_DELEGATION_TOKEN:
return new CreateDelegationTokenRequest(struct, apiVersion); return CreateDelegationTokenRequest.parse(buffer, apiVersion);
case RENEW_DELEGATION_TOKEN: case RENEW_DELEGATION_TOKEN:
return new RenewDelegationTokenRequest(struct, apiVersion); return RenewDelegationTokenRequest.parse(buffer, apiVersion);
case EXPIRE_DELEGATION_TOKEN: case EXPIRE_DELEGATION_TOKEN:
return new ExpireDelegationTokenRequest(struct, apiVersion); return ExpireDelegationTokenRequest.parse(buffer, apiVersion);
case DESCRIBE_DELEGATION_TOKEN: case DESCRIBE_DELEGATION_TOKEN:
return new DescribeDelegationTokenRequest(struct, apiVersion); return DescribeDelegationTokenRequest.parse(buffer, apiVersion);
case DELETE_GROUPS: case DELETE_GROUPS:
return new DeleteGroupsRequest(struct, apiVersion); return DeleteGroupsRequest.parse(buffer, apiVersion);
case ELECT_LEADERS: case ELECT_LEADERS:
return new ElectLeadersRequest(struct, apiVersion); return ElectLeadersRequest.parse(buffer, apiVersion);
case INCREMENTAL_ALTER_CONFIGS: case INCREMENTAL_ALTER_CONFIGS:
return new IncrementalAlterConfigsRequest(struct, apiVersion); return IncrementalAlterConfigsRequest.parse(buffer, apiVersion);
case ALTER_PARTITION_REASSIGNMENTS: case ALTER_PARTITION_REASSIGNMENTS:
return new AlterPartitionReassignmentsRequest(struct, apiVersion); return AlterPartitionReassignmentsRequest.parse(buffer, apiVersion);
case LIST_PARTITION_REASSIGNMENTS: case LIST_PARTITION_REASSIGNMENTS:
return new ListPartitionReassignmentsRequest(struct, apiVersion); return ListPartitionReassignmentsRequest.parse(buffer, apiVersion);
case OFFSET_DELETE: case OFFSET_DELETE:
return new OffsetDeleteRequest(struct, apiVersion); return OffsetDeleteRequest.parse(buffer, apiVersion);
case DESCRIBE_CLIENT_QUOTAS: case DESCRIBE_CLIENT_QUOTAS:
return new DescribeClientQuotasRequest(struct, apiVersion); return DescribeClientQuotasRequest.parse(buffer, apiVersion);
case ALTER_CLIENT_QUOTAS: case ALTER_CLIENT_QUOTAS:
return new AlterClientQuotasRequest(struct, apiVersion); return AlterClientQuotasRequest.parse(buffer, apiVersion);
case DESCRIBE_USER_SCRAM_CREDENTIALS: case DESCRIBE_USER_SCRAM_CREDENTIALS:
return new DescribeUserScramCredentialsRequest(struct, apiVersion); return DescribeUserScramCredentialsRequest.parse(buffer, apiVersion);
case ALTER_USER_SCRAM_CREDENTIALS: case ALTER_USER_SCRAM_CREDENTIALS:
return new AlterUserScramCredentialsRequest(struct, apiVersion); return AlterUserScramCredentialsRequest.parse(buffer, apiVersion);
case VOTE: case VOTE:
return new VoteRequest(struct, apiVersion); return VoteRequest.parse(buffer, apiVersion);
case BEGIN_QUORUM_EPOCH: case BEGIN_QUORUM_EPOCH:
return new BeginQuorumEpochRequest(struct, apiVersion); return BeginQuorumEpochRequest.parse(buffer, apiVersion);
case END_QUORUM_EPOCH: case END_QUORUM_EPOCH:
return new EndQuorumEpochRequest(struct, apiVersion); return EndQuorumEpochRequest.parse(buffer, apiVersion);
case DESCRIBE_QUORUM: case DESCRIBE_QUORUM:
return new DescribeQuorumRequest(struct, apiVersion); return DescribeQuorumRequest.parse(buffer, apiVersion);
case ALTER_ISR: case ALTER_ISR:
return new AlterIsrRequest(new AlterIsrRequestData(struct, apiVersion), apiVersion); return AlterIsrRequest.parse(buffer, apiVersion);
case UPDATE_FEATURES: case UPDATE_FEATURES:
return new UpdateFeaturesRequest(struct, apiVersion); return UpdateFeaturesRequest.parse(buffer, apiVersion);
case ENVELOPE: case ENVELOPE:
return new EnvelopeRequest(struct, apiVersion); return EnvelopeRequest.parse(buffer, apiVersion);
default: default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " +
"code should be updated to do so.", apiKey)); "code should be updated to do so.", apiKey));

View File

@ -16,6 +16,4 @@
*/ */
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
public interface AbstractRequestResponse { public interface AbstractRequestResponse { }
}

View File

@ -16,38 +16,49 @@
*/ */
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.EnvelopeResponseData;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.AlterIsrResponseData;
import org.apache.kafka.common.message.ProduceResponseData;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.SendBuilder;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
public abstract class AbstractResponse implements AbstractRequestResponse { public abstract class AbstractResponse implements AbstractRequestResponse {
public static final int DEFAULT_THROTTLE_TIME = 0; public static final int DEFAULT_THROTTLE_TIME = 0;
protected Send toSend(String destination, ResponseHeader header, short apiVersion) { private final ApiKeys apiKey;
return new NetworkSend(destination, RequestUtils.serialize(header.toStruct(), toStruct(apiVersion)));
protected AbstractResponse(ApiKeys apiKey) {
this.apiKey = apiKey;
}
public final Send toSend(String destination, ResponseHeader header, short version) {
return SendBuilder.buildResponseSend(destination, header, data(), version);
} }
/** /**
* Visible for testing, typically {@link #toSend(String, ResponseHeader, short)} should be used instead. * Visible for testing, typically {@link #toSend(String, ResponseHeader, short)} should be used instead.
*/ */
public ByteBuffer serialize(ApiKeys apiKey, short version, int correlationId) { public final ByteBuffer serializeWithHeader(short version, int correlationId) {
ResponseHeader header = return serializeWithHeader(new ResponseHeader(correlationId, apiKey.responseHeaderVersion(version)), version);
new ResponseHeader(correlationId, apiKey.responseHeaderVersion(version)); }
return RequestUtils.serialize(header.toStruct(), toStruct(version));
final ByteBuffer serializeWithHeader(ResponseHeader header, short version) {
Objects.requireNonNull(header, "header should not be null");
return RequestUtils.serialize(header.data(), header.headerVersion(), data(), version);
}
// Visible for testing
final ByteBuffer serializeBody(short version) {
return RequestUtils.serialize(null, (short) 0, data(), version);
} }
/** /**
@ -84,17 +95,18 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
errorCounts.put(error, count + 1); errorCounts.put(error, count + 1);
} }
protected abstract Struct toStruct(short version); protected abstract Message data();
/** /**
* Parse a response from the provided buffer. The buffer is expected to hold both * Parse a response from the provided buffer. The buffer is expected to hold both
* the {@link ResponseHeader} as well as the response payload. * the {@link ResponseHeader} as well as the response payload.
*/ */
public static AbstractResponse parseResponse(ByteBuffer byteBuffer, RequestHeader requestHeader) { public static AbstractResponse parseResponse(ByteBuffer buffer, RequestHeader requestHeader) {
ApiKeys apiKey = requestHeader.apiKey(); ApiKeys apiKey = requestHeader.apiKey();
short apiVersion = requestHeader.apiVersion(); short apiVersion = requestHeader.apiVersion();
ResponseHeader responseHeader = ResponseHeader.parse(byteBuffer, apiKey.responseHeaderVersion(apiVersion)); ResponseHeader responseHeader = ResponseHeader.parse(buffer, apiKey.responseHeaderVersion(apiVersion));
if (requestHeader.correlationId() != responseHeader.correlationId()) { if (requestHeader.correlationId() != responseHeader.correlationId()) {
throw new CorrelationIdMismatchException("Correlation id for response (" throw new CorrelationIdMismatchException("Correlation id for response ("
+ responseHeader.correlationId() + ") does not match request (" + responseHeader.correlationId() + ") does not match request ("
@ -102,130 +114,129 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
requestHeader.correlationId(), responseHeader.correlationId()); requestHeader.correlationId(), responseHeader.correlationId());
} }
Struct struct = apiKey.parseResponse(apiVersion, byteBuffer); return AbstractResponse.parseResponse(apiKey, buffer, apiVersion);
return AbstractResponse.parseResponse(apiKey, struct, apiVersion);
} }
public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, short version) { public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer responseBuffer, short version) {
switch (apiKey) { switch (apiKey) {
case PRODUCE: case PRODUCE:
return new ProduceResponse(new ProduceResponseData(struct, version)); return ProduceResponse.parse(responseBuffer, version);
case FETCH: case FETCH:
return new FetchResponse<>(new FetchResponseData(struct, version)); return FetchResponse.parse(responseBuffer, version);
case LIST_OFFSETS: case LIST_OFFSETS:
return new ListOffsetResponse(struct, version); return ListOffsetResponse.parse(responseBuffer, version);
case METADATA: case METADATA:
return new MetadataResponse(struct, version); return MetadataResponse.parse(responseBuffer, version);
case OFFSET_COMMIT: case OFFSET_COMMIT:
return new OffsetCommitResponse(struct, version); return OffsetCommitResponse.parse(responseBuffer, version);
case OFFSET_FETCH: case OFFSET_FETCH:
return new OffsetFetchResponse(struct, version); return OffsetFetchResponse.parse(responseBuffer, version);
case FIND_COORDINATOR: case FIND_COORDINATOR:
return new FindCoordinatorResponse(struct, version); return FindCoordinatorResponse.parse(responseBuffer, version);
case JOIN_GROUP: case JOIN_GROUP:
return new JoinGroupResponse(struct, version); return JoinGroupResponse.parse(responseBuffer, version);
case HEARTBEAT: case HEARTBEAT:
return new HeartbeatResponse(struct, version); return HeartbeatResponse.parse(responseBuffer, version);
case LEAVE_GROUP: case LEAVE_GROUP:
return new LeaveGroupResponse(struct, version); return LeaveGroupResponse.parse(responseBuffer, version);
case SYNC_GROUP: case SYNC_GROUP:
return new SyncGroupResponse(struct, version); return SyncGroupResponse.parse(responseBuffer, version);
case STOP_REPLICA: case STOP_REPLICA:
return new StopReplicaResponse(struct, version); return StopReplicaResponse.parse(responseBuffer, version);
case CONTROLLED_SHUTDOWN: case CONTROLLED_SHUTDOWN:
return new ControlledShutdownResponse(struct, version); return ControlledShutdownResponse.parse(responseBuffer, version);
case UPDATE_METADATA: case UPDATE_METADATA:
return new UpdateMetadataResponse(struct, version); return UpdateMetadataResponse.parse(responseBuffer, version);
case LEADER_AND_ISR: case LEADER_AND_ISR:
return new LeaderAndIsrResponse(struct, version); return LeaderAndIsrResponse.parse(responseBuffer, version);
case DESCRIBE_GROUPS: case DESCRIBE_GROUPS:
return new DescribeGroupsResponse(struct, version); return DescribeGroupsResponse.parse(responseBuffer, version);
case LIST_GROUPS: case LIST_GROUPS:
return new ListGroupsResponse(struct, version); return ListGroupsResponse.parse(responseBuffer, version);
case SASL_HANDSHAKE: case SASL_HANDSHAKE:
return new SaslHandshakeResponse(struct, version); return SaslHandshakeResponse.parse(responseBuffer, version);
case API_VERSIONS: case API_VERSIONS:
return ApiVersionsResponse.fromStruct(struct, version); return ApiVersionsResponse.parse(responseBuffer, version);
case CREATE_TOPICS: case CREATE_TOPICS:
return new CreateTopicsResponse(struct, version); return CreateTopicsResponse.parse(responseBuffer, version);
case DELETE_TOPICS: case DELETE_TOPICS:
return new DeleteTopicsResponse(struct, version); return DeleteTopicsResponse.parse(responseBuffer, version);
case DELETE_RECORDS: case DELETE_RECORDS:
return new DeleteRecordsResponse(struct, version); return DeleteRecordsResponse.parse(responseBuffer, version);
case INIT_PRODUCER_ID: case INIT_PRODUCER_ID:
return new InitProducerIdResponse(struct, version); return InitProducerIdResponse.parse(responseBuffer, version);
case OFFSET_FOR_LEADER_EPOCH: case OFFSET_FOR_LEADER_EPOCH:
return new OffsetsForLeaderEpochResponse(struct, version); return OffsetsForLeaderEpochResponse.parse(responseBuffer, version);
case ADD_PARTITIONS_TO_TXN: case ADD_PARTITIONS_TO_TXN:
return new AddPartitionsToTxnResponse(struct, version); return AddPartitionsToTxnResponse.parse(responseBuffer, version);
case ADD_OFFSETS_TO_TXN: case ADD_OFFSETS_TO_TXN:
return new AddOffsetsToTxnResponse(struct, version); return AddOffsetsToTxnResponse.parse(responseBuffer, version);
case END_TXN: case END_TXN:
return new EndTxnResponse(struct, version); return EndTxnResponse.parse(responseBuffer, version);
case WRITE_TXN_MARKERS: case WRITE_TXN_MARKERS:
return new WriteTxnMarkersResponse(struct, version); return WriteTxnMarkersResponse.parse(responseBuffer, version);
case TXN_OFFSET_COMMIT: case TXN_OFFSET_COMMIT:
return new TxnOffsetCommitResponse(struct, version); return TxnOffsetCommitResponse.parse(responseBuffer, version);
case DESCRIBE_ACLS: case DESCRIBE_ACLS:
return new DescribeAclsResponse(struct, version); return DescribeAclsResponse.parse(responseBuffer, version);
case CREATE_ACLS: case CREATE_ACLS:
return new CreateAclsResponse(struct, version); return CreateAclsResponse.parse(responseBuffer, version);
case DELETE_ACLS: case DELETE_ACLS:
return new DeleteAclsResponse(struct, version); return DeleteAclsResponse.parse(responseBuffer, version);
case DESCRIBE_CONFIGS: case DESCRIBE_CONFIGS:
return new DescribeConfigsResponse(struct, version); return DescribeConfigsResponse.parse(responseBuffer, version);
case ALTER_CONFIGS: case ALTER_CONFIGS:
return new AlterConfigsResponse(struct, version); return AlterConfigsResponse.parse(responseBuffer, version);
case ALTER_REPLICA_LOG_DIRS: case ALTER_REPLICA_LOG_DIRS:
return new AlterReplicaLogDirsResponse(struct, version); return AlterReplicaLogDirsResponse.parse(responseBuffer, version);
case DESCRIBE_LOG_DIRS: case DESCRIBE_LOG_DIRS:
return new DescribeLogDirsResponse(struct, version); return DescribeLogDirsResponse.parse(responseBuffer, version);
case SASL_AUTHENTICATE: case SASL_AUTHENTICATE:
return new SaslAuthenticateResponse(struct, version); return SaslAuthenticateResponse.parse(responseBuffer, version);
case CREATE_PARTITIONS: case CREATE_PARTITIONS:
return new CreatePartitionsResponse(struct, version); return CreatePartitionsResponse.parse(responseBuffer, version);
case CREATE_DELEGATION_TOKEN: case CREATE_DELEGATION_TOKEN:
return new CreateDelegationTokenResponse(struct, version); return CreateDelegationTokenResponse.parse(responseBuffer, version);
case RENEW_DELEGATION_TOKEN: case RENEW_DELEGATION_TOKEN:
return new RenewDelegationTokenResponse(struct, version); return RenewDelegationTokenResponse.parse(responseBuffer, version);
case EXPIRE_DELEGATION_TOKEN: case EXPIRE_DELEGATION_TOKEN:
return new ExpireDelegationTokenResponse(struct, version); return ExpireDelegationTokenResponse.parse(responseBuffer, version);
case DESCRIBE_DELEGATION_TOKEN: case DESCRIBE_DELEGATION_TOKEN:
return new DescribeDelegationTokenResponse(struct, version); return DescribeDelegationTokenResponse.parse(responseBuffer, version);
case DELETE_GROUPS: case DELETE_GROUPS:
return new DeleteGroupsResponse(struct, version); return DeleteGroupsResponse.parse(responseBuffer, version);
case ELECT_LEADERS: case ELECT_LEADERS:
return new ElectLeadersResponse(struct, version); return ElectLeadersResponse.parse(responseBuffer, version);
case INCREMENTAL_ALTER_CONFIGS: case INCREMENTAL_ALTER_CONFIGS:
return new IncrementalAlterConfigsResponse(struct, version); return IncrementalAlterConfigsResponse.parse(responseBuffer, version);
case ALTER_PARTITION_REASSIGNMENTS: case ALTER_PARTITION_REASSIGNMENTS:
return new AlterPartitionReassignmentsResponse(struct, version); return AlterPartitionReassignmentsResponse.parse(responseBuffer, version);
case LIST_PARTITION_REASSIGNMENTS: case LIST_PARTITION_REASSIGNMENTS:
return new ListPartitionReassignmentsResponse(struct, version); return ListPartitionReassignmentsResponse.parse(responseBuffer, version);
case OFFSET_DELETE: case OFFSET_DELETE:
return new OffsetDeleteResponse(struct, version); return OffsetDeleteResponse.parse(responseBuffer, version);
case DESCRIBE_CLIENT_QUOTAS: case DESCRIBE_CLIENT_QUOTAS:
return new DescribeClientQuotasResponse(struct, version); return DescribeClientQuotasResponse.parse(responseBuffer, version);
case ALTER_CLIENT_QUOTAS: case ALTER_CLIENT_QUOTAS:
return new AlterClientQuotasResponse(struct, version); return AlterClientQuotasResponse.parse(responseBuffer, version);
case DESCRIBE_USER_SCRAM_CREDENTIALS: case DESCRIBE_USER_SCRAM_CREDENTIALS:
return new DescribeUserScramCredentialsResponse(struct, version); return DescribeUserScramCredentialsResponse.parse(responseBuffer, version);
case ALTER_USER_SCRAM_CREDENTIALS: case ALTER_USER_SCRAM_CREDENTIALS:
return new AlterUserScramCredentialsResponse(struct, version); return AlterUserScramCredentialsResponse.parse(responseBuffer, version);
case VOTE: case VOTE:
return new VoteResponse(struct, version); return VoteResponse.parse(responseBuffer, version);
case BEGIN_QUORUM_EPOCH: case BEGIN_QUORUM_EPOCH:
return new BeginQuorumEpochResponse(struct, version); return BeginQuorumEpochResponse.parse(responseBuffer, version);
case END_QUORUM_EPOCH: case END_QUORUM_EPOCH:
return new EndQuorumEpochResponse(struct, version); return EndQuorumEpochResponse.parse(responseBuffer, version);
case DESCRIBE_QUORUM: case DESCRIBE_QUORUM:
return new DescribeQuorumResponse(struct, version); return DescribeQuorumResponse.parse(responseBuffer, version);
case ALTER_ISR: case ALTER_ISR:
return new AlterIsrResponse(new AlterIsrResponseData(struct, version)); return AlterIsrResponse.parse(responseBuffer, version);
case UPDATE_FEATURES: case UPDATE_FEATURES:
return new UpdateFeaturesResponse(struct, version); return UpdateFeaturesResponse.parse(responseBuffer, version);
case ENVELOPE: case ENVELOPE:
return new EnvelopeResponse(new EnvelopeResponseData(struct, version)); return EnvelopeResponse.parse(responseBuffer, version);
default: default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " +
"code should be updated to do so.", apiKey)); "code should be updated to do so.", apiKey));
@ -241,11 +252,13 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
return false; return false;
} }
public int throttleTimeMs() { public ApiKeys apiKey() {
return DEFAULT_THROTTLE_TIME; return apiKey;
} }
public String toString(short version) { public abstract int throttleTimeMs();
return toStruct(version).toString();
public String toString() {
return data().toString();
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AddOffsetsToTxnRequestData; import org.apache.kafka.common.message.AddOffsetsToTxnRequestData;
import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.message.AddOffsetsToTxnResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -52,14 +52,9 @@ public class AddOffsetsToTxnRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public AddOffsetsToTxnRequest(Struct struct, short version) {
super(ApiKeys.ADD_OFFSETS_TO_TXN, version);
this.data = new AddOffsetsToTxnRequestData(struct, version);
}
@Override @Override
protected Struct toStruct() { protected AddOffsetsToTxnRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -70,6 +65,6 @@ public class AddOffsetsToTxnRequest extends AbstractRequest {
} }
public static AddOffsetsToTxnRequest parse(ByteBuffer buffer, short version) { public static AddOffsetsToTxnRequest parse(ByteBuffer buffer, short version) {
return new AddOffsetsToTxnRequest(ApiKeys.ADD_OFFSETS_TO_TXN.parseRequest(version, buffer), version); return new AddOffsetsToTxnRequest(new AddOffsetsToTxnRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.message.AddOffsetsToTxnResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -42,30 +42,27 @@ public class AddOffsetsToTxnResponse extends AbstractResponse {
public AddOffsetsToTxnResponseData data; public AddOffsetsToTxnResponseData data;
public AddOffsetsToTxnResponse(AddOffsetsToTxnResponseData data) { public AddOffsetsToTxnResponse(AddOffsetsToTxnResponseData data) {
super(ApiKeys.ADD_OFFSETS_TO_TXN);
this.data = data; this.data = data;
} }
public AddOffsetsToTxnResponse(Struct struct, short version) {
this.data = new AddOffsetsToTxnResponseData(struct, version);
}
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
return errorCounts(Errors.forCode(data.errorCode())); return errorCounts(Errors.forCode(data.errorCode()));
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
} }
@Override
protected AddOffsetsToTxnResponseData data() {
return data;
}
public static AddOffsetsToTxnResponse parse(ByteBuffer buffer, short version) { public static AddOffsetsToTxnResponse parse(ByteBuffer buffer, short version) {
return new AddOffsetsToTxnResponse(ApiKeys.ADD_OFFSETS_TO_TXN.parseResponse(version, buffer), version); return new AddOffsetsToTxnResponse(new AddOffsetsToTxnResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -21,8 +21,8 @@ import org.apache.kafka.common.message.AddPartitionsToTxnRequestData;
import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopic; import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopic;
import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopicCollection; import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopicCollection;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -103,11 +103,6 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public AddPartitionsToTxnRequest(Struct struct, short version) {
super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
this.data = new AddPartitionsToTxnRequestData(struct, version);
}
public List<TopicPartition> partitions() { public List<TopicPartition> partitions() {
if (cachedPartitions != null) { if (cachedPartitions != null) {
return cachedPartitions; return cachedPartitions;
@ -117,8 +112,8 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
} }
@Override @Override
protected Struct toStruct() { protected AddPartitionsToTxnRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -131,6 +126,6 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
} }
public static AddPartitionsToTxnRequest parse(ByteBuffer buffer, short version) { public static AddPartitionsToTxnRequest parse(ByteBuffer buffer, short version) {
return new AddPartitionsToTxnRequest(ApiKeys.ADD_PARTITIONS_TO_TXN.parseRequest(version, buffer), version); return new AddPartitionsToTxnRequest(new AddPartitionsToTxnRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -23,8 +23,8 @@ import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartiti
import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnTopicResult; import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnTopicResult;
import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnTopicResultCollection; import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnTopicResultCollection;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -50,11 +50,13 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
private Map<TopicPartition, Errors> cachedErrorsMap = null; private Map<TopicPartition, Errors> cachedErrorsMap = null;
public AddPartitionsToTxnResponse(Struct struct, short version) { public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
this.data = new AddPartitionsToTxnResponseData(struct, version); super(ApiKeys.ADD_PARTITIONS_TO_TXN);
this.data = data;
} }
public AddPartitionsToTxnResponse(int throttleTimeMs, Map<TopicPartition, Errors> errors) { public AddPartitionsToTxnResponse(int throttleTimeMs, Map<TopicPartition, Errors> errors) {
super(ApiKeys.ADD_PARTITIONS_TO_TXN);
Map<String, AddPartitionsToTxnPartitionResultCollection> resultMap = new HashMap<>(); Map<String, AddPartitionsToTxnPartitionResultCollection> resultMap = new HashMap<>();
@ -115,12 +117,12 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected AddPartitionsToTxnResponseData data() {
return data.toStruct(version); return data;
} }
public static AddPartitionsToTxnResponse parse(ByteBuffer buffer, short version) { public static AddPartitionsToTxnResponse parse(ByteBuffer buffer, short version) {
return new AddPartitionsToTxnResponse(ApiKeys.ADD_PARTITIONS_TO_TXN.parseResponse(version, buffer), version); return new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -20,11 +20,13 @@ import org.apache.kafka.common.message.AlterClientQuotasRequestData;
import org.apache.kafka.common.message.AlterClientQuotasRequestData.EntityData; import org.apache.kafka.common.message.AlterClientQuotasRequestData.EntityData;
import org.apache.kafka.common.message.AlterClientQuotasRequestData.EntryData; import org.apache.kafka.common.message.AlterClientQuotasRequestData.EntryData;
import org.apache.kafka.common.message.AlterClientQuotasRequestData.OpData; import org.apache.kafka.common.message.AlterClientQuotasRequestData.OpData;
import org.apache.kafka.common.message.AlterClientQuotasResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaAlteration;
import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.quota.ClientQuotaEntity;
import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
@ -85,12 +87,7 @@ public class AlterClientQuotasRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public AlterClientQuotasRequest(Struct struct, short version) { public List<ClientQuotaAlteration> entries() {
super(ApiKeys.ALTER_CLIENT_QUOTAS, version);
this.data = new AlterClientQuotasRequestData(struct, version);
}
public Collection<ClientQuotaAlteration> entries() {
List<ClientQuotaAlteration> entries = new ArrayList<>(data.entries().size()); List<ClientQuotaAlteration> entries = new ArrayList<>(data.entries().size());
for (EntryData entryData : data.entries()) { for (EntryData entryData : data.entries()) {
Map<String, String> entity = new HashMap<>(entryData.entity().size()); Map<String, String> entity = new HashMap<>(entryData.entity().size());
@ -114,20 +111,29 @@ public class AlterClientQuotasRequest extends AbstractRequest {
} }
@Override @Override
public AlterClientQuotasResponse getErrorResponse(int throttleTimeMs, Throwable e) { protected AlterClientQuotasRequestData data() {
ArrayList<ClientQuotaEntity> entities = new ArrayList<>(data.entries().size()); return data;
for (EntryData entryData : data.entries()) {
Map<String, String> entity = new HashMap<>(entryData.entity().size());
for (EntityData entityData : entryData.entity()) {
entity.put(entityData.entityType(), entityData.entityName());
}
entities.add(new ClientQuotaEntity(entity));
}
return new AlterClientQuotasResponse(entities, throttleTimeMs, e);
} }
@Override @Override
protected Struct toStruct() { public AlterClientQuotasResponse getErrorResponse(int throttleTimeMs, Throwable e) {
return data.toStruct(version()); List<AlterClientQuotasResponseData.EntryData> responseEntries = new ArrayList<>();
for (EntryData entryData : data.entries()) {
List<AlterClientQuotasResponseData.EntityData> responseEntities = new ArrayList<>();
for (EntityData entityData : entryData.entity()) {
responseEntities.add(new AlterClientQuotasResponseData.EntityData()
.setEntityType(entityData.entityType())
.setEntityName(entityData.entityName()));
}
responseEntries.add(new AlterClientQuotasResponseData.EntryData().setEntity(responseEntities));
}
AlterClientQuotasResponseData responseData = new AlterClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setEntries(responseEntries);
return new AlterClientQuotasResponse(responseData);
}
public static AlterClientQuotasRequest parse(ByteBuffer buffer, short version) {
return new AlterClientQuotasRequest(new AlterClientQuotasRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -21,13 +21,12 @@ import org.apache.kafka.common.message.AlterClientQuotasResponseData;
import org.apache.kafka.common.message.AlterClientQuotasResponseData.EntityData; import org.apache.kafka.common.message.AlterClientQuotasResponseData.EntityData;
import org.apache.kafka.common.message.AlterClientQuotasResponseData.EntryData; import org.apache.kafka.common.message.AlterClientQuotasResponseData.EntryData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.quota.ClientQuotaEntity;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -36,39 +35,9 @@ public class AlterClientQuotasResponse extends AbstractResponse {
private final AlterClientQuotasResponseData data; private final AlterClientQuotasResponseData data;
public AlterClientQuotasResponse(Map<ClientQuotaEntity, ApiError> result, int throttleTimeMs) { public AlterClientQuotasResponse(AlterClientQuotasResponseData data) {
List<EntryData> entries = new ArrayList<>(result.size()); super(ApiKeys.ALTER_CLIENT_QUOTAS);
for (Map.Entry<ClientQuotaEntity, ApiError> entry : result.entrySet()) { this.data = data;
ApiError e = entry.getValue();
entries.add(new EntryData()
.setErrorCode(e.error().code())
.setErrorMessage(e.message())
.setEntity(toEntityData(entry.getKey())));
}
this.data = new AlterClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setEntries(entries);
}
public AlterClientQuotasResponse(Collection<ClientQuotaEntity> entities, int throttleTimeMs, Throwable e) {
ApiError apiError = ApiError.fromThrowable(e);
List<EntryData> entries = new ArrayList<>(entities.size());
for (ClientQuotaEntity entity : entities) {
entries.add(new EntryData()
.setErrorCode(apiError.error().code())
.setErrorMessage(apiError.message())
.setEntity(toEntityData(entity)));
}
this.data = new AlterClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setEntries(entries);
}
public AlterClientQuotasResponse(Struct struct, short version) {
this.data = new AlterClientQuotasResponseData(struct, version);
} }
public void complete(Map<ClientQuotaEntity, KafkaFutureImpl<Void>> futures) { public void complete(Map<ClientQuotaEntity, KafkaFutureImpl<Void>> futures) {
@ -108,8 +77,8 @@ public class AlterClientQuotasResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected AlterClientQuotasResponseData data() {
return data.toStruct(version); return data;
} }
private static List<EntityData> toEntityData(ClientQuotaEntity entity) { private static List<EntityData> toEntityData(ClientQuotaEntity entity) {
@ -123,6 +92,22 @@ public class AlterClientQuotasResponse extends AbstractResponse {
} }
public static AlterClientQuotasResponse parse(ByteBuffer buffer, short version) { public static AlterClientQuotasResponse parse(ByteBuffer buffer, short version) {
return new AlterClientQuotasResponse(ApiKeys.ALTER_CLIENT_QUOTAS.parseResponse(version, buffer), version); return new AlterClientQuotasResponse(new AlterClientQuotasResponseData(new ByteBufferAccessor(buffer), version));
} }
public static AlterClientQuotasResponse fromQuotaEntities(Map<ClientQuotaEntity, ApiError> result, int throttleTimeMs) {
List<EntryData> entries = new ArrayList<>(result.size());
for (Map.Entry<ClientQuotaEntity, ApiError> entry : result.entrySet()) {
ApiError e = entry.getValue();
entries.add(new EntryData()
.setErrorCode(e.error().code())
.setErrorMessage(e.message())
.setEntity(toEntityData(entry.getKey())));
}
return new AlterClientQuotasResponse(new AlterClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setEntries(entries));
}
} }

View File

@ -21,7 +21,7 @@ import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.message.AlterConfigsRequestData; import org.apache.kafka.common.message.AlterConfigsRequestData;
import org.apache.kafka.common.message.AlterConfigsResponseData; import org.apache.kafka.common.message.AlterConfigsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collection; import java.util.Collection;
@ -97,11 +97,6 @@ public class AlterConfigsRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public AlterConfigsRequest(Struct struct, short version) {
super(ApiKeys.ALTER_CONFIGS, version);
this.data = new AlterConfigsRequestData(struct, version);
}
public Map<ConfigResource, Config> configs() { public Map<ConfigResource, Config> configs() {
return data.resources().stream().collect(Collectors.toMap( return data.resources().stream().collect(Collectors.toMap(
resource -> new ConfigResource( resource -> new ConfigResource(
@ -117,8 +112,8 @@ public class AlterConfigsRequest extends AbstractRequest {
} }
@Override @Override
protected Struct toStruct() { protected AlterConfigsRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -138,6 +133,6 @@ public class AlterConfigsRequest extends AbstractRequest {
} }
public static AlterConfigsRequest parse(ByteBuffer buffer, short version) { public static AlterConfigsRequest parse(ByteBuffer buffer, short version) {
return new AlterConfigsRequest(ApiKeys.ALTER_CONFIGS.parseRequest(version, buffer), version); return new AlterConfigsRequest(new AlterConfigsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -20,8 +20,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.message.AlterConfigsResponseData; import org.apache.kafka.common.message.AlterConfigsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -32,17 +32,10 @@ public class AlterConfigsResponse extends AbstractResponse {
private final AlterConfigsResponseData data; private final AlterConfigsResponseData data;
public AlterConfigsResponse(AlterConfigsResponseData data) { public AlterConfigsResponse(AlterConfigsResponseData data) {
super(ApiKeys.ALTER_CONFIGS);
this.data = data; this.data = data;
} }
public AlterConfigsResponse(Struct struct, short version) {
this.data = new AlterConfigsResponseData(struct, version);
}
public AlterConfigsResponseData data() {
return data;
}
public Map<ConfigResource, ApiError> errors() { public Map<ConfigResource, ApiError> errors() {
return data.responses().stream().collect(Collectors.toMap( return data.responses().stream().collect(Collectors.toMap(
response -> new ConfigResource( response -> new ConfigResource(
@ -63,12 +56,12 @@ public class AlterConfigsResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { public AlterConfigsResponseData data() {
return data.toStruct(version); return data;
} }
public static AlterConfigsResponse parse(ByteBuffer buffer, short version) { public static AlterConfigsResponse parse(ByteBuffer buffer, short version) {
return new AlterConfigsResponse(ApiKeys.ALTER_CONFIGS.parseResponse(version, buffer), version); return new AlterConfigsResponse(new AlterConfigsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -20,8 +20,10 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AlterIsrRequestData; import org.apache.kafka.common.message.AlterIsrRequestData;
import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.AlterIsrResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
public class AlterIsrRequest extends AbstractRequest { public class AlterIsrRequest extends AbstractRequest {
@ -36,11 +38,6 @@ public class AlterIsrRequest extends AbstractRequest {
return data; return data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
/** /**
* Get an error response for a request with specified throttle time in the response if applicable * Get an error response for a request with specified throttle time in the response if applicable
*/ */
@ -51,6 +48,10 @@ public class AlterIsrRequest extends AbstractRequest {
.setErrorCode(Errors.forException(e).code())); .setErrorCode(Errors.forException(e).code()));
} }
public static AlterIsrRequest parse(ByteBuffer buffer, short version) {
return new AlterIsrRequest(new AlterIsrRequestData(new ByteBufferAccessor(buffer), version), version);
}
public static class Builder extends AbstractRequest.Builder<AlterIsrRequest> { public static class Builder extends AbstractRequest.Builder<AlterIsrRequest> {
private final AlterIsrRequestData data; private final AlterIsrRequestData data;

View File

@ -18,9 +18,11 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.AlterIsrResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
@ -29,6 +31,7 @@ public class AlterIsrResponse extends AbstractResponse {
private final AlterIsrResponseData data; private final AlterIsrResponseData data;
public AlterIsrResponse(AlterIsrResponseData data) { public AlterIsrResponse(AlterIsrResponseData data) {
super(ApiKeys.ALTER_ISR);
this.data = data; this.data = data;
} }
@ -46,13 +49,12 @@ public class AlterIsrResponse extends AbstractResponse {
return counts; return counts;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
} }
public static AlterIsrResponse parse(ByteBuffer buffer, short version) {
return new AlterIsrResponse(new AlterIsrResponseData(new ByteBufferAccessor(buffer), version));
}
} }

View File

@ -23,7 +23,7 @@ import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData;
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse;
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -52,39 +52,21 @@ public class AlterPartitionReassignmentsRequest extends AbstractRequest {
} }
private final AlterPartitionReassignmentsRequestData data; private final AlterPartitionReassignmentsRequestData data;
private final short version;
private AlterPartitionReassignmentsRequest(AlterPartitionReassignmentsRequestData data, short version) { private AlterPartitionReassignmentsRequest(AlterPartitionReassignmentsRequestData data, short version) {
super(ApiKeys.ALTER_PARTITION_REASSIGNMENTS, version); super(ApiKeys.ALTER_PARTITION_REASSIGNMENTS, version);
this.data = data; this.data = data;
this.version = version;
}
AlterPartitionReassignmentsRequest(Struct struct, short version) {
super(ApiKeys.ALTER_PARTITION_REASSIGNMENTS, version);
this.data = new AlterPartitionReassignmentsRequestData(struct, version);
this.version = version;
} }
public static AlterPartitionReassignmentsRequest parse(ByteBuffer buffer, short version) { public static AlterPartitionReassignmentsRequest parse(ByteBuffer buffer, short version) {
return new AlterPartitionReassignmentsRequest( return new AlterPartitionReassignmentsRequest(new AlterPartitionReassignmentsRequestData(
ApiKeys.ALTER_PARTITION_REASSIGNMENTS.parseRequest(version, buffer), new ByteBufferAccessor(buffer), version), version);
version
);
} }
public AlterPartitionReassignmentsRequestData data() { public AlterPartitionReassignmentsRequestData data() {
return data; return data;
} }
/**
* Visible for testing.
*/
@Override
public Struct toStruct() {
return data.toStruct(version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
ApiError apiError = ApiError.fromThrowable(e); ApiError apiError = ApiError.fromThrowable(e);

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -30,20 +30,14 @@ public class AlterPartitionReassignmentsResponse extends AbstractResponse {
private final AlterPartitionReassignmentsResponseData data; private final AlterPartitionReassignmentsResponseData data;
public AlterPartitionReassignmentsResponse(Struct struct) {
this(struct, ApiKeys.ALTER_PARTITION_REASSIGNMENTS.latestVersion());
}
public AlterPartitionReassignmentsResponse(AlterPartitionReassignmentsResponseData data) { public AlterPartitionReassignmentsResponse(AlterPartitionReassignmentsResponseData data) {
super(ApiKeys.ALTER_PARTITION_REASSIGNMENTS);
this.data = data; this.data = data;
} }
AlterPartitionReassignmentsResponse(Struct struct, short version) {
this.data = new AlterPartitionReassignmentsResponseData(struct, version);
}
public static AlterPartitionReassignmentsResponse parse(ByteBuffer buffer, short version) { public static AlterPartitionReassignmentsResponse parse(ByteBuffer buffer, short version) {
return new AlterPartitionReassignmentsResponse(ApiKeys.ALTER_PARTITION_REASSIGNMENTS.responseSchema(version).read(buffer), version); return new AlterPartitionReassignmentsResponse(
new AlterPartitionReassignmentsResponseData(new ByteBufferAccessor(buffer), version));
} }
public AlterPartitionReassignmentsResponseData data() { public AlterPartitionReassignmentsResponseData data() {
@ -71,9 +65,4 @@ public class AlterPartitionReassignmentsResponse extends AbstractResponse {
)); ));
return counts; return counts;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
} }

View File

@ -17,18 +17,20 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData; import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData;
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData;
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
public class AlterReplicaLogDirsRequest extends AbstractRequest { public class AlterReplicaLogDirsRequest extends AbstractRequest {
@ -53,22 +55,16 @@ public class AlterReplicaLogDirsRequest extends AbstractRequest {
} }
} }
public AlterReplicaLogDirsRequest(Struct struct, short version) {
super(ApiKeys.ALTER_REPLICA_LOG_DIRS, version);
this.data = new AlterReplicaLogDirsRequestData(struct, version);
}
public AlterReplicaLogDirsRequest(AlterReplicaLogDirsRequestData data, short version) { public AlterReplicaLogDirsRequest(AlterReplicaLogDirsRequestData data, short version) {
super(ApiKeys.ALTER_REPLICA_LOG_DIRS, version); super(ApiKeys.ALTER_REPLICA_LOG_DIRS, version);
this.data = data; this.data = data;
} }
@Override @Override
protected Struct toStruct() { protected AlterReplicaLogDirsRequestData data() {
return data.toStruct(version()); return data;
} }
@Override
public AlterReplicaLogDirsResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AlterReplicaLogDirsResponse getErrorResponse(int throttleTimeMs, Throwable e) {
AlterReplicaLogDirsResponseData data = new AlterReplicaLogDirsResponseData(); AlterReplicaLogDirsResponseData data = new AlterReplicaLogDirsResponseData();
data.setResults(this.data.dirs().stream().flatMap(alterDir -> data.setResults(this.data.dirs().stream().flatMap(alterDir ->
@ -93,6 +89,6 @@ public class AlterReplicaLogDirsRequest extends AbstractRequest {
} }
public static AlterReplicaLogDirsRequest parse(ByteBuffer buffer, short version) { public static AlterReplicaLogDirsRequest parse(ByteBuffer buffer, short version) {
return new AlterReplicaLogDirsRequest(ApiKeys.ALTER_REPLICA_LOG_DIRS.parseRequest(version, buffer), version); return new AlterReplicaLogDirsRequest(new AlterReplicaLogDirsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -17,15 +17,15 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
/** /**
* Possible error codes: * Possible error codes:
* *
@ -38,23 +38,16 @@ public class AlterReplicaLogDirsResponse extends AbstractResponse {
private final AlterReplicaLogDirsResponseData data; private final AlterReplicaLogDirsResponseData data;
public AlterReplicaLogDirsResponse(Struct struct, short version) {
this.data = new AlterReplicaLogDirsResponseData(struct, version);
}
public AlterReplicaLogDirsResponse(AlterReplicaLogDirsResponseData data) { public AlterReplicaLogDirsResponse(AlterReplicaLogDirsResponseData data) {
super(ApiKeys.ALTER_REPLICA_LOG_DIRS);
this.data = data; this.data = data;
} }
@Override
public AlterReplicaLogDirsResponseData data() { public AlterReplicaLogDirsResponseData data() {
return data; return data;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -70,7 +63,7 @@ public class AlterReplicaLogDirsResponse extends AbstractResponse {
} }
public static AlterReplicaLogDirsResponse parse(ByteBuffer buffer, short version) { public static AlterReplicaLogDirsResponse parse(ByteBuffer buffer, short version) {
return new AlterReplicaLogDirsResponse(ApiKeys.ALTER_REPLICA_LOG_DIRS.responseSchema(version).read(buffer), version); return new AlterReplicaLogDirsResponse(new AlterReplicaLogDirsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -19,7 +19,7 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData; import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
@ -48,39 +48,21 @@ public class AlterUserScramCredentialsRequest extends AbstractRequest {
} }
} }
private AlterUserScramCredentialsRequestData data; private final AlterUserScramCredentialsRequestData data;
private final short version;
private AlterUserScramCredentialsRequest(AlterUserScramCredentialsRequestData data, short version) { private AlterUserScramCredentialsRequest(AlterUserScramCredentialsRequestData data, short version) {
super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS, version); super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS, version);
this.data = data; this.data = data;
this.version = version;
}
AlterUserScramCredentialsRequest(Struct struct, short version) {
super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS, version);
this.data = new AlterUserScramCredentialsRequestData(struct, version);
this.version = version;
} }
public static AlterUserScramCredentialsRequest parse(ByteBuffer buffer, short version) { public static AlterUserScramCredentialsRequest parse(ByteBuffer buffer, short version) {
return new AlterUserScramCredentialsRequest( return new AlterUserScramCredentialsRequest(new AlterUserScramCredentialsRequestData(new ByteBufferAccessor(buffer), version), version);
ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.parseRequest(version, buffer), version
);
} }
public AlterUserScramCredentialsRequestData data() { public AlterUserScramCredentialsRequestData data() {
return data; return data;
} }
/**
* Visible for testing.
*/
@Override
public Struct toStruct() {
return data.toStruct(version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
ApiError apiError = ApiError.fromThrowable(e); ApiError apiError = ApiError.fromThrowable(e);

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -28,22 +28,11 @@ public class AlterUserScramCredentialsResponse extends AbstractResponse {
private final AlterUserScramCredentialsResponseData data; private final AlterUserScramCredentialsResponseData data;
public AlterUserScramCredentialsResponse(Struct struct) {
this(struct, ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.latestVersion());
}
public AlterUserScramCredentialsResponse(AlterUserScramCredentialsResponseData responseData) { public AlterUserScramCredentialsResponse(AlterUserScramCredentialsResponseData responseData) {
super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS);
this.data = responseData; this.data = responseData;
} }
AlterUserScramCredentialsResponse(Struct struct, short version) {
this.data = new AlterUserScramCredentialsResponseData(struct, version);
}
public static AlterUserScramCredentialsResponse parse(ByteBuffer buffer, short version) {
return new AlterUserScramCredentialsResponse(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.responseSchema(version).read(buffer), version);
}
public AlterUserScramCredentialsResponseData data() { public AlterUserScramCredentialsResponseData data() {
return data; return data;
} }
@ -63,8 +52,7 @@ public class AlterUserScramCredentialsResponse extends AbstractResponse {
return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode()))); return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode())));
} }
@Override public static AlterUserScramCredentialsResponse parse(ByteBuffer buffer, short version) {
protected Struct toStruct(short version) { return new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData(new ByteBufferAccessor(buffer), version));
return data.toStruct(version);
} }
} }

View File

@ -19,13 +19,9 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.util.Objects; import java.util.Objects;
import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
/** /**
* Encapsulates an error code (via the Errors enum) and an optional message. Generally, the optional message is only * Encapsulates an error code (via the Errors enum) and an optional message. Generally, the optional message is only
* defined if it adds information over the default message associated with the error code. * defined if it adds information over the default message associated with the error code.
@ -47,12 +43,6 @@ public class ApiError {
return new ApiError(error, message); return new ApiError(error, message);
} }
public ApiError(Struct struct) {
error = Errors.forCode(struct.get(ERROR_CODE));
// In some cases, the error message field was introduced in newer version
message = struct.getOrElse(ERROR_MESSAGE, null);
}
public ApiError(Errors error) { public ApiError(Errors error) {
this(error, error.message()); this(error, error.message());
} }
@ -67,12 +57,6 @@ public class ApiError {
this.message = message; this.message = message;
} }
public void write(Struct struct) {
struct.set(ERROR_CODE, error.code());
if (error != Errors.NONE)
struct.setIfExists(ERROR_MESSAGE, message);
}
public boolean is(Errors error) { public boolean is(Errors error) {
return this.error == error; return this.error == error;
} }

View File

@ -22,8 +22,8 @@ import org.apache.kafka.common.message.ApiVersionsResponseData;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.AppInfoParser;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -60,7 +60,7 @@ public class ApiVersionsRequest extends AbstractRequest {
private final Short unsupportedRequestVersion; private final Short unsupportedRequestVersion;
public final ApiVersionsRequestData data; private final ApiVersionsRequestData data;
public ApiVersionsRequest(ApiVersionsRequestData data, short version) { public ApiVersionsRequest(ApiVersionsRequestData data, short version) {
this(data, version, null); this(data, version, null);
@ -78,10 +78,6 @@ public class ApiVersionsRequest extends AbstractRequest {
this.unsupportedRequestVersion = unsupportedRequestVersion; this.unsupportedRequestVersion = unsupportedRequestVersion;
} }
public ApiVersionsRequest(Struct struct, short version) {
this(new ApiVersionsRequestData(struct, version), version);
}
public boolean hasUnsupportedRequestVersion() { public boolean hasUnsupportedRequestVersion() {
return unsupportedRequestVersion != null; return unsupportedRequestVersion != null;
} }
@ -96,8 +92,8 @@ public class ApiVersionsRequest extends AbstractRequest {
} }
@Override @Override
protected Struct toStruct() { public ApiVersionsRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -124,7 +120,7 @@ public class ApiVersionsRequest extends AbstractRequest {
} }
public static ApiVersionsRequest parse(ByteBuffer buffer, short version) { public static ApiVersionsRequest parse(ByteBuffer buffer, short version) {
return new ApiVersionsRequest(ApiKeys.API_VERSIONS.parseRequest(version, buffer), version); return new ApiVersionsRequest(new ApiVersionsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -29,8 +29,6 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureK
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.SchemaException;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -45,38 +43,21 @@ public class ApiVersionsResponse extends AbstractResponse {
public static final long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1L; public static final long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1L;
public static final ApiVersionsResponse DEFAULT_API_VERSIONS_RESPONSE = public static final ApiVersionsResponse DEFAULT_API_VERSIONS_RESPONSE = createApiVersionsResponse(
createApiVersionsResponse( DEFAULT_THROTTLE_TIME, RecordBatch.CURRENT_MAGIC_VALUE);
DEFAULT_THROTTLE_TIME,
RecordBatch.CURRENT_MAGIC_VALUE,
Features.emptySupportedFeatures(),
Features.emptyFinalizedFeatures(),
UNKNOWN_FINALIZED_FEATURES_EPOCH
);
public final ApiVersionsResponseData data; public final ApiVersionsResponseData data;
public ApiVersionsResponse(ApiVersionsResponseData data) { public ApiVersionsResponse(ApiVersionsResponseData data) {
super(ApiKeys.API_VERSIONS);
this.data = data; this.data = data;
} }
public ApiVersionsResponse(Struct struct) { @Override
this(new ApiVersionsResponseData(struct, (short) (ApiVersionsResponseData.SCHEMAS.length - 1)));
}
public ApiVersionsResponse(Struct struct, short version) {
this(new ApiVersionsResponseData(struct, version));
}
public ApiVersionsResponseData data() { public ApiVersionsResponseData data() {
return data; return data;
} }
@Override
protected Struct toStruct(short version) {
return this.data.toStruct(version);
}
public ApiVersionsResponseKey apiVersion(short apiKey) { public ApiVersionsResponseKey apiVersion(short apiKey) {
return data.apiKeys().find(apiKey); return data.apiKeys().find(apiKey);
} }
@ -100,48 +81,23 @@ public class ApiVersionsResponse extends AbstractResponse {
// Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest
// using a version higher than that supported by the broker, a version 0 response is sent // using a version higher than that supported by the broker, a version 0 response is sent
// to the client indicating UNSUPPORTED_VERSION. When the client receives the response, it // to the client indicating UNSUPPORTED_VERSION. When the client receives the response, it
// falls back while parsing it into a Struct which means that the version received by this // falls back while parsing it which means that the version received by this
// method is not necessary the real one. It may be version 0 as well. // method is not necessarily the real one. It may be version 0 as well.
int prev = buffer.position(); int prev = buffer.position();
try { try {
return new ApiVersionsResponse( return new ApiVersionsResponse(new ApiVersionsResponseData(new ByteBufferAccessor(buffer), version));
new ApiVersionsResponseData(new ByteBufferAccessor(buffer), version));
} catch (RuntimeException e) { } catch (RuntimeException e) {
buffer.position(prev); buffer.position(prev);
if (version != 0) if (version != 0)
return new ApiVersionsResponse( return new ApiVersionsResponse(new ApiVersionsResponseData(new ByteBufferAccessor(buffer), (short) 0));
new ApiVersionsResponseData(new ByteBufferAccessor(buffer), (short) 0));
else else
throw e; throw e;
} }
} }
public static ApiVersionsResponse fromStruct(Struct struct, short version) { public static ApiVersionsResponse createApiVersionsResponse(final int throttleTimeMs, final byte minMagic) {
// Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest return createApiVersionsResponse(throttleTimeMs, minMagic, Features.emptySupportedFeatures(),
// using a version higher than that supported by the broker, a version 0 response is sent Features.emptyFinalizedFeatures(), UNKNOWN_FINALIZED_FEATURES_EPOCH);
// to the client indicating UNSUPPORTED_VERSION. When the client receives the response, it
// falls back while parsing it into a Struct which means that the version received by this
// method is not necessary the real one. It may be version 0 as well.
try {
return new ApiVersionsResponse(struct, version);
} catch (SchemaException e) {
if (version != 0)
return new ApiVersionsResponse(struct, (short) 0);
else
throw e;
}
}
public static ApiVersionsResponse createApiVersionsResponse(
final int throttleTimeMs,
final byte minMagic) {
return createApiVersionsResponse(
throttleTimeMs,
minMagic,
Features.emptySupportedFeatures(),
Features.emptyFinalizedFeatures(),
UNKNOWN_FINALIZED_FEATURES_EPOCH
);
} }
private static ApiVersionsResponse createApiVersionsResponse( private static ApiVersionsResponse createApiVersionsResponse(

View File

@ -20,9 +20,10 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
public class BeginQuorumEpochRequest extends AbstractRequest { public class BeginQuorumEpochRequest extends AbstractRequest {
@ -52,14 +53,9 @@ public class BeginQuorumEpochRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public BeginQuorumEpochRequest(Struct struct, short version) {
super(ApiKeys.BEGIN_QUORUM_EPOCH, version);
this.data = new BeginQuorumEpochRequestData(struct, version);
}
@Override @Override
protected Struct toStruct() { protected BeginQuorumEpochRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -68,6 +64,10 @@ public class BeginQuorumEpochRequest extends AbstractRequest {
.setErrorCode(Errors.forException(e).code())); .setErrorCode(Errors.forException(e).code()));
} }
public static BeginQuorumEpochRequest parse(ByteBuffer buffer, short version) {
return new BeginQuorumEpochRequest(new BeginQuorumEpochRequestData(new ByteBufferAccessor(buffer), version), version);
}
public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition, public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition,
int leaderEpoch, int leaderEpoch,
int leaderId) { int leaderId) {

View File

@ -20,8 +20,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -45,18 +45,10 @@ public class BeginQuorumEpochResponse extends AbstractResponse {
public final BeginQuorumEpochResponseData data; public final BeginQuorumEpochResponseData data;
public BeginQuorumEpochResponse(BeginQuorumEpochResponseData data) { public BeginQuorumEpochResponse(BeginQuorumEpochResponseData data) {
super(ApiKeys.BEGIN_QUORUM_EPOCH);
this.data = data; this.data = data;
} }
public BeginQuorumEpochResponse(Struct struct, short version) {
this.data = new BeginQuorumEpochResponseData(struct, version);
}
public BeginQuorumEpochResponse(Struct struct) {
short latestVersion = (short) (BeginQuorumEpochResponseData.SCHEMAS.length - 1);
this.data = new BeginQuorumEpochResponseData(struct, latestVersion);
}
public static BeginQuorumEpochResponseData singletonResponse( public static BeginQuorumEpochResponseData singletonResponse(
Errors topLevelError, Errors topLevelError,
TopicPartition topicPartition, TopicPartition topicPartition,
@ -78,11 +70,6 @@ public class BeginQuorumEpochResponse extends AbstractResponse {
); );
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errors = new HashMap<>(); Map<Errors, Integer> errors = new HashMap<>();
@ -98,8 +85,18 @@ public class BeginQuorumEpochResponse extends AbstractResponse {
return errors; return errors;
} }
public static BeginQuorumEpochResponse parse(ByteBuffer buffer, short version) { @Override
return new BeginQuorumEpochResponse(ApiKeys.BEGIN_QUORUM_EPOCH.responseSchema(version).read(buffer), version); protected BeginQuorumEpochResponseData data() {
return data;
} }
} @Override
public int throttleTimeMs() {
return DEFAULT_THROTTLE_TIME;
}
public static BeginQuorumEpochResponse parse(ByteBuffer buffer, short version) {
return new BeginQuorumEpochResponse(new BeginQuorumEpochResponseData(new ByteBufferAccessor(buffer), version));
}
}

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.ControlledShutdownRequestData; import org.apache.kafka.common.message.ControlledShutdownRequestData;
import org.apache.kafka.common.message.ControlledShutdownResponseData; import org.apache.kafka.common.message.ControlledShutdownResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -47,18 +47,10 @@ public class ControlledShutdownRequest extends AbstractRequest {
} }
private final ControlledShutdownRequestData data; private final ControlledShutdownRequestData data;
private final short version;
private ControlledShutdownRequest(ControlledShutdownRequestData data, short version) { private ControlledShutdownRequest(ControlledShutdownRequestData data, short version) {
super(ApiKeys.CONTROLLED_SHUTDOWN, version); super(ApiKeys.CONTROLLED_SHUTDOWN, version);
this.data = data; this.data = data;
this.version = version;
}
public ControlledShutdownRequest(Struct struct, short version) {
super(ApiKeys.CONTROLLED_SHUTDOWN, version);
this.data = new ControlledShutdownRequestData(struct, version);
this.version = version;
} }
@Override @Override
@ -69,13 +61,8 @@ public class ControlledShutdownRequest extends AbstractRequest {
} }
public static ControlledShutdownRequest parse(ByteBuffer buffer, short version) { public static ControlledShutdownRequest parse(ByteBuffer buffer, short version) {
return new ControlledShutdownRequest( return new ControlledShutdownRequest(new ControlledShutdownRequestData(new ByteBufferAccessor(buffer), version),
ApiKeys.CONTROLLED_SHUTDOWN.parseRequest(version, buffer), version); version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version);
} }
public ControlledShutdownRequestData data() { public ControlledShutdownRequestData data() {

View File

@ -20,8 +20,8 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.ControlledShutdownResponseData; import org.apache.kafka.common.message.ControlledShutdownResponseData;
import org.apache.kafka.common.message.ControlledShutdownResponseData.RemainingPartition; import org.apache.kafka.common.message.ControlledShutdownResponseData.RemainingPartition;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -40,13 +40,10 @@ public class ControlledShutdownResponse extends AbstractResponse {
private final ControlledShutdownResponseData data; private final ControlledShutdownResponseData data;
public ControlledShutdownResponse(ControlledShutdownResponseData data) { public ControlledShutdownResponse(ControlledShutdownResponseData data) {
super(ApiKeys.CONTROLLED_SHUTDOWN);
this.data = data; this.data = data;
} }
public ControlledShutdownResponse(Struct struct, short version) {
this.data = new ControlledShutdownResponseData(struct, version);
}
public Errors error() { public Errors error() {
return Errors.forCode(data.errorCode()); return Errors.forCode(data.errorCode());
} }
@ -56,13 +53,13 @@ public class ControlledShutdownResponse extends AbstractResponse {
return errorCounts(error()); return errorCounts(error());
} }
public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) { @Override
return new ControlledShutdownResponse(ApiKeys.CONTROLLED_SHUTDOWN.parseResponse(version, buffer), version); public int throttleTimeMs() {
return DEFAULT_THROTTLE_TIME;
} }
@Override public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) {
protected Struct toStruct(short version) { return new ControlledShutdownResponse(new ControlledShutdownResponseData(new ByteBufferAccessor(buffer), version));
return data.toStruct(version);
} }
public ControlledShutdownResponseData data() { public ControlledShutdownResponseData data() {

View File

@ -21,15 +21,15 @@ import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.CreateAclsRequestData; import org.apache.kafka.common.message.CreateAclsRequestData;
import org.apache.kafka.common.message.CreateAclsRequestData.AclCreation; import org.apache.kafka.common.message.CreateAclsRequestData.AclCreation;
import org.apache.kafka.common.message.CreateAclsResponseData; import org.apache.kafka.common.message.CreateAclsResponseData;
import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult; import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -48,7 +48,7 @@ public class CreateAclsRequest extends AbstractRequest {
@Override @Override
public CreateAclsRequest build(short version) { public CreateAclsRequest build(short version) {
return new CreateAclsRequest(version, data); return new CreateAclsRequest(data, version);
} }
@Override @Override
@ -59,23 +59,19 @@ public class CreateAclsRequest extends AbstractRequest {
private final CreateAclsRequestData data; private final CreateAclsRequestData data;
CreateAclsRequest(short version, CreateAclsRequestData data) { CreateAclsRequest(CreateAclsRequestData data, short version) {
super(ApiKeys.CREATE_ACLS, version); super(ApiKeys.CREATE_ACLS, version);
validate(data); validate(data);
this.data = data; this.data = data;
} }
public CreateAclsRequest(Struct struct, short version) {
this(version, new CreateAclsRequestData(struct, version));
}
public List<AclCreation> aclCreations() { public List<AclCreation> aclCreations() {
return data.creations(); return data.creations();
} }
@Override @Override
protected Struct toStruct() { protected CreateAclsRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -88,7 +84,7 @@ public class CreateAclsRequest extends AbstractRequest {
} }
public static CreateAclsRequest parse(ByteBuffer buffer, short version) { public static CreateAclsRequest parse(ByteBuffer buffer, short version) {
return new CreateAclsRequest(ApiKeys.CREATE_ACLS.parseRequest(version, buffer), version); return new CreateAclsRequest(new CreateAclsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
private void validate(CreateAclsRequestData data) { private void validate(CreateAclsRequestData data) {

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.CreateAclsResponseData; import org.apache.kafka.common.message.CreateAclsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
@ -29,16 +29,13 @@ public class CreateAclsResponse extends AbstractResponse {
private final CreateAclsResponseData data; private final CreateAclsResponseData data;
public CreateAclsResponse(CreateAclsResponseData data) { public CreateAclsResponse(CreateAclsResponseData data) {
super(ApiKeys.CREATE_ACLS);
this.data = data; this.data = data;
} }
public CreateAclsResponse(Struct struct, short version) {
this.data = new CreateAclsResponseData(struct, version);
}
@Override @Override
protected Struct toStruct(short version) { protected CreateAclsResponseData data() {
return data.toStruct(version); return data;
} }
@Override @Override
@ -56,7 +53,7 @@ public class CreateAclsResponse extends AbstractResponse {
} }
public static CreateAclsResponse parse(ByteBuffer buffer, short version) { public static CreateAclsResponse parse(ByteBuffer buffer, short version) {
return new CreateAclsResponse(ApiKeys.CREATE_ACLS.responseSchema(version).read(buffer), version); return new CreateAclsResponse(new CreateAclsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.CreateDelegationTokenRequestData; import org.apache.kafka.common.message.CreateDelegationTokenRequestData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.KafkaPrincipal;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -33,18 +33,9 @@ public class CreateDelegationTokenRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public CreateDelegationTokenRequest(Struct struct, short version) {
super(ApiKeys.CREATE_DELEGATION_TOKEN, version);
this.data = new CreateDelegationTokenRequestData(struct, version);
}
public static CreateDelegationTokenRequest parse(ByteBuffer buffer, short version) { public static CreateDelegationTokenRequest parse(ByteBuffer buffer, short version) {
return new CreateDelegationTokenRequest(ApiKeys.CREATE_DELEGATION_TOKEN.parseRequest(version, buffer), version); return new CreateDelegationTokenRequest(new CreateDelegationTokenRequestData(new ByteBufferAccessor(buffer), version),
} version);
@Override
protected Struct toStruct() {
return data.toStruct(version());
} }
public CreateDelegationTokenRequestData data() { public CreateDelegationTokenRequestData data() {

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.CreateDelegationTokenResponseData; import org.apache.kafka.common.message.CreateDelegationTokenResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.KafkaPrincipal;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -30,15 +30,13 @@ public class CreateDelegationTokenResponse extends AbstractResponse {
private final CreateDelegationTokenResponseData data; private final CreateDelegationTokenResponseData data;
public CreateDelegationTokenResponse(CreateDelegationTokenResponseData data) { public CreateDelegationTokenResponse(CreateDelegationTokenResponseData data) {
super(ApiKeys.CREATE_DELEGATION_TOKEN);
this.data = data; this.data = data;
} }
public CreateDelegationTokenResponse(Struct struct, short version) {
this.data = new CreateDelegationTokenResponseData(struct, version);
}
public static CreateDelegationTokenResponse parse(ByteBuffer buffer, short version) { public static CreateDelegationTokenResponse parse(ByteBuffer buffer, short version) {
return new CreateDelegationTokenResponse(ApiKeys.CREATE_DELEGATION_TOKEN.responseSchema(version).read(buffer), version); return new CreateDelegationTokenResponse(
new CreateDelegationTokenResponseData(new ByteBufferAccessor(buffer), version));
} }
public static CreateDelegationTokenResponse prepareResponse(int throttleTimeMs, public static CreateDelegationTokenResponse prepareResponse(int throttleTimeMs,
@ -75,11 +73,6 @@ public class CreateDelegationTokenResponse extends AbstractResponse {
return errorCounts(error()); return errorCounts(error());
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();

View File

@ -22,7 +22,7 @@ import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartiti
import org.apache.kafka.common.message.CreatePartitionsResponseData; import org.apache.kafka.common.message.CreatePartitionsResponseData;
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -55,15 +55,6 @@ public class CreatePartitionsRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public CreatePartitionsRequest(Struct struct, short apiVersion) {
this(new CreatePartitionsRequestData(struct, apiVersion), apiVersion);
}
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
public CreatePartitionsRequestData data() { public CreatePartitionsRequestData data() {
return data; return data;
} }
@ -85,6 +76,6 @@ public class CreatePartitionsRequest extends AbstractRequest {
} }
public static CreatePartitionsRequest parse(ByteBuffer buffer, short version) { public static CreatePartitionsRequest parse(ByteBuffer buffer, short version) {
return new CreatePartitionsRequest(ApiKeys.CREATE_PARTITIONS.parseRequest(version, buffer), version); return new CreatePartitionsRequest(new CreatePartitionsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -19,35 +19,26 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.CreatePartitionsResponseData; import org.apache.kafka.common.message.CreatePartitionsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
public class CreatePartitionsResponse extends AbstractResponse { public class CreatePartitionsResponse extends AbstractResponse {
private final CreatePartitionsResponseData data; private final CreatePartitionsResponseData data;
public CreatePartitionsResponse(CreatePartitionsResponseData data) { public CreatePartitionsResponse(CreatePartitionsResponseData data) {
super(ApiKeys.CREATE_PARTITIONS);
this.data = data; this.data = data;
} }
public CreatePartitionsResponse(Struct struct, short version) {
this.data = new CreatePartitionsResponseData(struct, version);
}
public CreatePartitionsResponseData data() { public CreatePartitionsResponseData data() {
return data; return data;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> counts = new HashMap<>(); Map<Errors, Integer> counts = new HashMap<>();
@ -58,7 +49,7 @@ public class CreatePartitionsResponse extends AbstractResponse {
} }
public static CreatePartitionsResponse parse(ByteBuffer buffer, short version) { public static CreatePartitionsResponse parse(ByteBuffer buffer, short version) {
return new CreatePartitionsResponse(ApiKeys.CREATE_PARTITIONS.parseResponse(version, buffer), version); return new CreatePartitionsResponse(new CreatePartitionsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -25,7 +25,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData;
import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
public class CreateTopicsRequest extends AbstractRequest { public class CreateTopicsRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<CreateTopicsRequest> { public static class Builder extends AbstractRequest.Builder<CreateTopicsRequest> {
@ -72,16 +72,11 @@ public class CreateTopicsRequest extends AbstractRequest {
public static final int NO_NUM_PARTITIONS = -1; public static final int NO_NUM_PARTITIONS = -1;
public static final short NO_REPLICATION_FACTOR = -1; public static final short NO_REPLICATION_FACTOR = -1;
private CreateTopicsRequest(CreateTopicsRequestData data, short version) { public CreateTopicsRequest(CreateTopicsRequestData data, short version) {
super(ApiKeys.CREATE_TOPICS, version); super(ApiKeys.CREATE_TOPICS, version);
this.data = data; this.data = data;
} }
public CreateTopicsRequest(Struct struct, short version) {
super(ApiKeys.CREATE_TOPICS, version);
this.data = new CreateTopicsRequestData(struct, version);
}
public CreateTopicsRequestData data() { public CreateTopicsRequestData data() {
return data; return data;
} }
@ -103,14 +98,6 @@ public class CreateTopicsRequest extends AbstractRequest {
} }
public static CreateTopicsRequest parse(ByteBuffer buffer, short version) { public static CreateTopicsRequest parse(ByteBuffer buffer, short version) {
return new CreateTopicsRequest(ApiKeys.CREATE_TOPICS.parseRequest(version, buffer), version); return new CreateTopicsRequest(new CreateTopicsRequestData(new ByteBufferAccessor(buffer), version), version);
}
/**
* Visible for testing.
*/
@Override
public Struct toStruct() {
return data.toStruct(version());
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -46,22 +46,14 @@ public class CreateTopicsResponse extends AbstractResponse {
private final CreateTopicsResponseData data; private final CreateTopicsResponseData data;
public CreateTopicsResponse(CreateTopicsResponseData data) { public CreateTopicsResponse(CreateTopicsResponseData data) {
super(ApiKeys.CREATE_TOPICS);
this.data = data; this.data = data;
} }
public CreateTopicsResponse(Struct struct, short version) {
this.data = new CreateTopicsResponseData(struct, version);
}
public CreateTopicsResponseData data() { public CreateTopicsResponseData data() {
return data; return data;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -77,8 +69,7 @@ public class CreateTopicsResponse extends AbstractResponse {
} }
public static CreateTopicsResponse parse(ByteBuffer buffer, short version) { public static CreateTopicsResponse parse(ByteBuffer buffer, short version) {
return new CreateTopicsResponse( return new CreateTopicsResponse(new CreateTopicsResponseData(new ByteBufferAccessor(buffer), version));
ApiKeys.CREATE_TOPICS.responseSchema(version).read(buffer), version);
} }
@Override @Override

View File

@ -28,7 +28,7 @@ import org.apache.kafka.common.message.DeleteAclsRequestData.DeleteAclsFilter;
import org.apache.kafka.common.message.DeleteAclsResponseData; import org.apache.kafka.common.message.DeleteAclsResponseData;
import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourcePatternFilter;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
@ -49,7 +49,7 @@ public class DeleteAclsRequest extends AbstractRequest {
@Override @Override
public DeleteAclsRequest build(short version) { public DeleteAclsRequest build(short version) {
return new DeleteAclsRequest(version, data); return new DeleteAclsRequest(data, version);
} }
@Override @Override
@ -61,7 +61,7 @@ public class DeleteAclsRequest extends AbstractRequest {
private final DeleteAclsRequestData data; private final DeleteAclsRequestData data;
private DeleteAclsRequest(short version, DeleteAclsRequestData data) { private DeleteAclsRequest(DeleteAclsRequestData data, short version) {
super(ApiKeys.DELETE_ACLS, version); super(ApiKeys.DELETE_ACLS, version);
this.data = data; this.data = data;
normalizeAndValidate(); normalizeAndValidate();
@ -95,18 +95,13 @@ public class DeleteAclsRequest extends AbstractRequest {
} }
} }
public DeleteAclsRequest(Struct struct, short version) {
super(ApiKeys.DELETE_ACLS, version);
this.data = new DeleteAclsRequestData(struct, version);
}
public List<AclBindingFilter> filters() { public List<AclBindingFilter> filters() {
return data.filters().stream().map(DeleteAclsRequest::aclBindingFilter).collect(Collectors.toList()); return data.filters().stream().map(DeleteAclsRequest::aclBindingFilter).collect(Collectors.toList());
} }
@Override @Override
protected Struct toStruct() { protected DeleteAclsRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -118,11 +113,11 @@ public class DeleteAclsRequest extends AbstractRequest {
.setErrorMessage(apiError.message())); .setErrorMessage(apiError.message()));
return new DeleteAclsResponse(new DeleteAclsResponseData() return new DeleteAclsResponse(new DeleteAclsResponseData()
.setThrottleTimeMs(throttleTimeMs) .setThrottleTimeMs(throttleTimeMs)
.setFilterResults(filterResults)); .setFilterResults(filterResults), version());
} }
public static DeleteAclsRequest parse(ByteBuffer buffer, short version) { public static DeleteAclsRequest parse(ByteBuffer buffer, short version) {
return new DeleteAclsRequest(DELETE_ACLS.parseRequest(version, buffer), version); return new DeleteAclsRequest(new DeleteAclsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
public static DeleteAclsFilter deleteAclsFilter(AclBindingFilter filter) { public static DeleteAclsFilter deleteAclsFilter(AclBindingFilter filter) {

View File

@ -18,17 +18,17 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.DeleteAclsResponseData; import org.apache.kafka.common.message.DeleteAclsResponseData;
import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult;
import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.authorizer.AclDeleteResult;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -44,18 +44,15 @@ public class DeleteAclsResponse extends AbstractResponse {
private final DeleteAclsResponseData data; private final DeleteAclsResponseData data;
public DeleteAclsResponse(DeleteAclsResponseData data) { public DeleteAclsResponse(DeleteAclsResponseData data, short version) {
super(ApiKeys.DELETE_ACLS);
this.data = data; this.data = data;
} validate(version);
public DeleteAclsResponse(Struct struct, short version) {
data = new DeleteAclsResponseData(struct, version);
} }
@Override @Override
protected Struct toStruct(short version) { protected DeleteAclsResponseData data() {
validate(version); return data;
return data.toStruct(version);
} }
@Override @Override
@ -73,7 +70,7 @@ public class DeleteAclsResponse extends AbstractResponse {
} }
public static DeleteAclsResponse parse(ByteBuffer buffer, short version) { public static DeleteAclsResponse parse(ByteBuffer buffer, short version) {
return new DeleteAclsResponse(ApiKeys.DELETE_ACLS.parseResponse(version, buffer), version); return new DeleteAclsResponse(new DeleteAclsResponseData(new ByteBufferAccessor(buffer), version), version);
} }
public String toString() { public String toString() {

View File

@ -21,8 +21,8 @@ import org.apache.kafka.common.message.DeleteGroupsResponseData;
import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult; import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult;
import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResultCollection; import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResultCollection;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -53,11 +53,6 @@ public class DeleteGroupsRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public DeleteGroupsRequest(Struct struct, short version) {
super(ApiKeys.DELETE_GROUPS, version);
this.data = new DeleteGroupsRequestData(struct, version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
Errors error = Errors.forException(e); Errors error = Errors.forException(e);
@ -76,11 +71,11 @@ public class DeleteGroupsRequest extends AbstractRequest {
} }
public static DeleteGroupsRequest parse(ByteBuffer buffer, short version) { public static DeleteGroupsRequest parse(ByteBuffer buffer, short version) {
return new DeleteGroupsRequest(ApiKeys.DELETE_GROUPS.parseRequest(version, buffer), version); return new DeleteGroupsRequest(new DeleteGroupsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override
protected Struct toStruct() { protected DeleteGroupsRequestData data() {
return data.toStruct(version()); return data;
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DeleteGroupsResponseData; import org.apache.kafka.common.message.DeleteGroupsResponseData;
import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult; import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -42,21 +42,13 @@ public class DeleteGroupsResponse extends AbstractResponse {
public final DeleteGroupsResponseData data; public final DeleteGroupsResponseData data;
public DeleteGroupsResponse(DeleteGroupsResponseData data) { public DeleteGroupsResponse(DeleteGroupsResponseData data) {
super(ApiKeys.DELETE_GROUPS);
this.data = data; this.data = data;
} }
public DeleteGroupsResponse(Struct struct) {
short latestVersion = (short) (DeleteGroupsResponseData.SCHEMAS.length - 1);
this.data = new DeleteGroupsResponseData(struct, latestVersion);
}
public DeleteGroupsResponse(Struct struct, short version) {
this.data = new DeleteGroupsResponseData(struct, version);
}
@Override @Override
protected Struct toStruct(short version) { protected DeleteGroupsResponseData data() {
return data.toStruct(version); return data;
} }
public Map<String, Errors> errors() { public Map<String, Errors> errors() {
@ -85,7 +77,7 @@ public class DeleteGroupsResponse extends AbstractResponse {
} }
public static DeleteGroupsResponse parse(ByteBuffer buffer, short version) { public static DeleteGroupsResponse parse(ByteBuffer buffer, short version) {
return new DeleteGroupsResponse(ApiKeys.DELETE_GROUPS.parseResponse(version, buffer), version); return new DeleteGroupsResponse(new DeleteGroupsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -22,8 +22,8 @@ import org.apache.kafka.common.message.DeleteRecordsRequestData.DeleteRecordsTop
import org.apache.kafka.common.message.DeleteRecordsResponseData; import org.apache.kafka.common.message.DeleteRecordsResponseData;
import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsTopicResult; import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsTopicResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -57,16 +57,7 @@ public class DeleteRecordsRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public DeleteRecordsRequest(Struct struct, short version) {
super(ApiKeys.DELETE_RECORDS, version);
this.data = new DeleteRecordsRequestData(struct, version);
}
@Override @Override
protected Struct toStruct() {
return data.toStruct(version());
}
public DeleteRecordsRequestData data() { public DeleteRecordsRequestData data() {
return data; return data;
} }
@ -89,6 +80,6 @@ public class DeleteRecordsRequest extends AbstractRequest {
} }
public static DeleteRecordsRequest parse(ByteBuffer buffer, short version) { public static DeleteRecordsRequest parse(ByteBuffer buffer, short version) {
return new DeleteRecordsRequest(ApiKeys.DELETE_RECORDS.parseRequest(version, buffer), version); return new DeleteRecordsRequest(new DeleteRecordsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DeleteRecordsResponseData; import org.apache.kafka.common.message.DeleteRecordsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -42,18 +42,10 @@ public class DeleteRecordsResponse extends AbstractResponse {
*/ */
public DeleteRecordsResponse(DeleteRecordsResponseData data) { public DeleteRecordsResponse(DeleteRecordsResponseData data) {
super(ApiKeys.DELETE_RECORDS);
this.data = data; this.data = data;
} }
public DeleteRecordsResponse(Struct struct, short version) {
this.data = new DeleteRecordsResponseData(struct, version);
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
public DeleteRecordsResponseData data() { public DeleteRecordsResponseData data() {
return data; return data;
} }
@ -75,7 +67,7 @@ public class DeleteRecordsResponse extends AbstractResponse {
} }
public static DeleteRecordsResponse parse(ByteBuffer buffer, short version) { public static DeleteRecordsResponse parse(ByteBuffer buffer, short version) {
return new DeleteRecordsResponse(ApiKeys.DELETE_RECORDS.parseResponse(version, buffer), version); return new DeleteRecordsResponse(new DeleteRecordsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -20,15 +20,12 @@ import org.apache.kafka.common.message.DeleteTopicsRequestData;
import org.apache.kafka.common.message.DeleteTopicsResponseData; import org.apache.kafka.common.message.DeleteTopicsResponseData;
import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicResult; import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class DeleteTopicsRequest extends AbstractRequest { public class DeleteTopicsRequest extends AbstractRequest {
private DeleteTopicsRequestData data;
private final short version;
public static class Builder extends AbstractRequest.Builder<DeleteTopicsRequest> { public static class Builder extends AbstractRequest.Builder<DeleteTopicsRequest> {
private DeleteTopicsRequestData data; private DeleteTopicsRequestData data;
@ -48,21 +45,11 @@ public class DeleteTopicsRequest extends AbstractRequest {
} }
} }
private DeleteTopicsRequestData data;
private DeleteTopicsRequest(DeleteTopicsRequestData data, short version) { private DeleteTopicsRequest(DeleteTopicsRequestData data, short version) {
super(ApiKeys.DELETE_TOPICS, version); super(ApiKeys.DELETE_TOPICS, version);
this.data = data; this.data = data;
this.version = version;
}
public DeleteTopicsRequest(Struct struct, short version) {
super(ApiKeys.DELETE_TOPICS, version);
this.data = new DeleteTopicsRequestData(struct, version);
this.version = version;
}
@Override
protected Struct toStruct() {
return data.toStruct(version);
} }
public DeleteTopicsRequestData data() { public DeleteTopicsRequestData data() {
@ -72,7 +59,7 @@ public class DeleteTopicsRequest extends AbstractRequest {
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
DeleteTopicsResponseData response = new DeleteTopicsResponseData(); DeleteTopicsResponseData response = new DeleteTopicsResponseData();
if (version >= 1) { if (version() >= 1) {
response.setThrottleTimeMs(throttleTimeMs); response.setThrottleTimeMs(throttleTimeMs);
} }
ApiError apiError = ApiError.fromThrowable(e); ApiError apiError = ApiError.fromThrowable(e);
@ -85,7 +72,7 @@ public class DeleteTopicsRequest extends AbstractRequest {
} }
public static DeleteTopicsRequest parse(ByteBuffer buffer, short version) { public static DeleteTopicsRequest parse(ByteBuffer buffer, short version) {
return new DeleteTopicsRequest(ApiKeys.DELETE_TOPICS.parseRequest(version, buffer), version); return new DeleteTopicsRequest(new DeleteTopicsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DeleteTopicsResponseData; import org.apache.kafka.common.message.DeleteTopicsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -41,18 +41,10 @@ public class DeleteTopicsResponse extends AbstractResponse {
private DeleteTopicsResponseData data; private DeleteTopicsResponseData data;
public DeleteTopicsResponse(DeleteTopicsResponseData data) { public DeleteTopicsResponse(DeleteTopicsResponseData data) {
super(ApiKeys.DELETE_TOPICS);
this.data = data; this.data = data;
} }
public DeleteTopicsResponse(Struct struct, short version) {
this.data = new DeleteTopicsResponseData(struct, version);
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -72,7 +64,7 @@ public class DeleteTopicsResponse extends AbstractResponse {
} }
public static DeleteTopicsResponse parse(ByteBuffer buffer, short version) { public static DeleteTopicsResponse parse(ByteBuffer buffer, short version) {
return new DeleteTopicsResponse(ApiKeys.DELETE_TOPICS.parseResponse(version, buffer), version); return new DeleteTopicsResponse(new DeleteTopicsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -24,7 +24,7 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.DescribeAclsRequestData; import org.apache.kafka.common.message.DescribeAclsRequestData;
import org.apache.kafka.common.message.DescribeAclsResponseData; import org.apache.kafka.common.message.DescribeAclsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourcePatternFilter;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
@ -89,20 +89,10 @@ public class DescribeAclsRequest extends AbstractRequest {
} }
} }
public DescribeAclsRequest(Struct struct, short version) {
super(ApiKeys.DESCRIBE_ACLS, version);
this.data = new DescribeAclsRequestData(struct, version);
}
public DescribeAclsRequestData data() { public DescribeAclsRequestData data() {
return data; return data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable throwable) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable throwable) {
ApiError error = ApiError.fromThrowable(throwable); ApiError error = ApiError.fromThrowable(throwable);
@ -110,11 +100,11 @@ public class DescribeAclsRequest extends AbstractRequest {
.setThrottleTimeMs(throttleTimeMs) .setThrottleTimeMs(throttleTimeMs)
.setErrorCode(error.error().code()) .setErrorCode(error.error().code())
.setErrorMessage(error.message()); .setErrorMessage(error.message());
return new DescribeAclsResponse(response); return new DescribeAclsResponse(response, version());
} }
public static DescribeAclsRequest parse(ByteBuffer buffer, short version) { public static DescribeAclsRequest parse(ByteBuffer buffer, short version) {
return new DescribeAclsRequest(ApiKeys.DESCRIBE_ACLS.parseRequest(version, buffer), version); return new DescribeAclsRequest(new DescribeAclsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
public AclBindingFilter filter() { public AclBindingFilter filter() {

View File

@ -17,6 +17,15 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
@ -24,40 +33,37 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.DescribeAclsResponseData; import org.apache.kafka.common.message.DescribeAclsResponseData;
import org.apache.kafka.common.message.DescribeAclsResponseData.AclDescription; import org.apache.kafka.common.message.DescribeAclsResponseData.AclDescription;
import org.apache.kafka.common.message.DescribeAclsResponseData.DescribeAclsResource; import org.apache.kafka.common.message.DescribeAclsResponseData.DescribeAclsResource;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
public class DescribeAclsResponse extends AbstractResponse { public class DescribeAclsResponse extends AbstractResponse {
private final DescribeAclsResponseData data; private final DescribeAclsResponseData data;
public DescribeAclsResponse(DescribeAclsResponseData data) { public DescribeAclsResponse(DescribeAclsResponseData data, short version) {
super(ApiKeys.DESCRIBE_ACLS);
this.data = data; this.data = data;
validate(Optional.of(version));
} }
public DescribeAclsResponse(Struct struct, short version) { // Skips version validation, visible for testing
this.data = new DescribeAclsResponseData(struct, version); DescribeAclsResponse(DescribeAclsResponseData data) {
super(ApiKeys.DESCRIBE_ACLS);
this.data = data;
validate(Optional.empty());
} }
@Override @Override
protected Struct toStruct(short version) { protected DescribeAclsResponseData data() {
validate(version); return data;
return data.toStruct(version);
} }
@Override @Override
@ -79,7 +85,7 @@ public class DescribeAclsResponse extends AbstractResponse {
} }
public static DescribeAclsResponse parse(ByteBuffer buffer, short version) { public static DescribeAclsResponse parse(ByteBuffer buffer, short version) {
return new DescribeAclsResponse(ApiKeys.DESCRIBE_ACLS.responseSchema(version).read(buffer), version); return new DescribeAclsResponse(new DescribeAclsResponseData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override
@ -87,8 +93,8 @@ public class DescribeAclsResponse extends AbstractResponse {
return version >= 1; return version >= 1;
} }
private void validate(short version) { private void validate(Optional<Short> version) {
if (version == 0) { if (version.isPresent() && version.get() == 0) {
final boolean unsupported = acls().stream() final boolean unsupported = acls().stream()
.anyMatch(acl -> acl.patternType() != PatternType.LITERAL.code()); .anyMatch(acl -> acl.patternType() != PatternType.LITERAL.code());
if (unsupported) { if (unsupported) {

View File

@ -18,11 +18,13 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeClientQuotasRequestData; import org.apache.kafka.common.message.DescribeClientQuotasRequestData;
import org.apache.kafka.common.message.DescribeClientQuotasRequestData.ComponentData; import org.apache.kafka.common.message.DescribeClientQuotasRequestData.ComponentData;
import org.apache.kafka.common.message.DescribeClientQuotasResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.quota.ClientQuotaFilter; import org.apache.kafka.common.quota.ClientQuotaFilter;
import org.apache.kafka.common.quota.ClientQuotaFilterComponent; import org.apache.kafka.common.quota.ClientQuotaFilterComponent;
import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
@ -77,11 +79,6 @@ public class DescribeClientQuotasRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public DescribeClientQuotasRequest(Struct struct, short version) {
super(ApiKeys.DESCRIBE_CLIENT_QUOTAS, version);
this.data = new DescribeClientQuotasRequestData(struct, version);
}
public ClientQuotaFilter filter() { public ClientQuotaFilter filter() {
List<ClientQuotaFilterComponent> components = new ArrayList<>(data.components().size()); List<ClientQuotaFilterComponent> components = new ArrayList<>(data.components().size());
for (ComponentData componentData : data.components()) { for (ComponentData componentData : data.components()) {
@ -109,12 +106,23 @@ public class DescribeClientQuotasRequest extends AbstractRequest {
} }
@Override @Override
public DescribeClientQuotasResponse getErrorResponse(int throttleTimeMs, Throwable e) { protected DescribeClientQuotasRequestData data() {
return new DescribeClientQuotasResponse(throttleTimeMs, e); return data;
} }
@Override @Override
protected Struct toStruct() { public DescribeClientQuotasResponse getErrorResponse(int throttleTimeMs, Throwable e) {
return data.toStruct(version()); ApiError error = ApiError.fromThrowable(e);
return new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode(error.error().code())
.setErrorMessage(error.message())
.setEntries(null));
} }
public static DescribeClientQuotasRequest parse(ByteBuffer buffer, short version) {
return new DescribeClientQuotasRequest(new DescribeClientQuotasRequestData(new ByteBufferAccessor(buffer), version),
version);
}
} }

View File

@ -22,8 +22,8 @@ import org.apache.kafka.common.message.DescribeClientQuotasResponseData.EntityDa
import org.apache.kafka.common.message.DescribeClientQuotasResponseData.EntryData; import org.apache.kafka.common.message.DescribeClientQuotasResponseData.EntryData;
import org.apache.kafka.common.message.DescribeClientQuotasResponseData.ValueData; import org.apache.kafka.common.message.DescribeClientQuotasResponseData.ValueData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.quota.ClientQuotaEntity;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -36,48 +36,9 @@ public class DescribeClientQuotasResponse extends AbstractResponse {
private final DescribeClientQuotasResponseData data; private final DescribeClientQuotasResponseData data;
public DescribeClientQuotasResponse(Map<ClientQuotaEntity, Map<String, Double>> entities, int throttleTimeMs) { public DescribeClientQuotasResponse(DescribeClientQuotasResponseData data) {
List<EntryData> entries = new ArrayList<>(entities.size()); super(ApiKeys.DESCRIBE_CLIENT_QUOTAS);
for (Map.Entry<ClientQuotaEntity, Map<String, Double>> entry : entities.entrySet()) { this.data = data;
ClientQuotaEntity quotaEntity = entry.getKey();
List<EntityData> entityData = new ArrayList<>(quotaEntity.entries().size());
for (Map.Entry<String, String> entityEntry : quotaEntity.entries().entrySet()) {
entityData.add(new EntityData()
.setEntityType(entityEntry.getKey())
.setEntityName(entityEntry.getValue()));
}
Map<String, Double> quotaValues = entry.getValue();
List<ValueData> valueData = new ArrayList<>(quotaValues.size());
for (Map.Entry<String, Double> valuesEntry : entry.getValue().entrySet()) {
valueData.add(new ValueData()
.setKey(valuesEntry.getKey())
.setValue(valuesEntry.getValue()));
}
entries.add(new EntryData()
.setEntity(entityData)
.setValues(valueData));
}
this.data = new DescribeClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode((short) 0)
.setErrorMessage(null)
.setEntries(entries);
}
public DescribeClientQuotasResponse(int throttleTimeMs, Throwable e) {
ApiError apiError = ApiError.fromThrowable(e);
this.data = new DescribeClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode(apiError.error().code())
.setErrorMessage(apiError.message())
.setEntries(null);
}
public DescribeClientQuotasResponse(Struct struct, short version) {
this.data = new DescribeClientQuotasResponseData(struct, version);
} }
public void complete(KafkaFutureImpl<Map<ClientQuotaEntity, Map<String, Double>>> future) { public void complete(KafkaFutureImpl<Map<ClientQuotaEntity, Map<String, Double>>> future) {
@ -109,17 +70,50 @@ public class DescribeClientQuotasResponse extends AbstractResponse {
return data.throttleTimeMs(); return data.throttleTimeMs();
} }
@Override
protected DescribeClientQuotasResponseData data() {
return data;
}
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
return errorCounts(Errors.forCode(data.errorCode())); return errorCounts(Errors.forCode(data.errorCode()));
} }
@Override public static DescribeClientQuotasResponse parse(ByteBuffer buffer, short version) {
protected Struct toStruct(short version) { return new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData(new ByteBufferAccessor(buffer), version));
return data.toStruct(version);
} }
public static DescribeClientQuotasResponse parse(ByteBuffer buffer, short version) { public static DescribeClientQuotasResponse fromQuotaEntities(Map<ClientQuotaEntity, Map<String, Double>> entities,
return new DescribeClientQuotasResponse(ApiKeys.DESCRIBE_CLIENT_QUOTAS.parseResponse(version, buffer), version); int throttleTimeMs) {
List<EntryData> entries = new ArrayList<>(entities.size());
for (Map.Entry<ClientQuotaEntity, Map<String, Double>> entry : entities.entrySet()) {
ClientQuotaEntity quotaEntity = entry.getKey();
List<EntityData> entityData = new ArrayList<>(quotaEntity.entries().size());
for (Map.Entry<String, String> entityEntry : quotaEntity.entries().entrySet()) {
entityData.add(new EntityData()
.setEntityType(entityEntry.getKey())
.setEntityName(entityEntry.getValue()));
}
Map<String, Double> quotaValues = entry.getValue();
List<ValueData> valueData = new ArrayList<>(quotaValues.size());
for (Map.Entry<String, Double> valuesEntry : entry.getValue().entrySet()) {
valueData.add(new ValueData()
.setKey(valuesEntry.getKey())
.setValue(valuesEntry.getValue()));
}
entries.add(new EntryData()
.setEntity(entityData)
.setValues(valueData));
}
return new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode((short) 0)
.setErrorMessage(null)
.setEntries(entries));
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeConfigsRequestData; import org.apache.kafka.common.message.DescribeConfigsRequestData;
import org.apache.kafka.common.message.DescribeConfigsResponseData; import org.apache.kafka.common.message.DescribeConfigsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -48,20 +48,11 @@ public class DescribeConfigsRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public DescribeConfigsRequest(Struct struct, short version) { @Override
super(ApiKeys.DESCRIBE_CONFIGS, version);
this.data = new DescribeConfigsRequestData(struct, version);
}
public DescribeConfigsRequestData data() { public DescribeConfigsRequestData data() {
return data; return data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
@Override @Override
public DescribeConfigsResponse getErrorResponse(int throttleTimeMs, Throwable e) { public DescribeConfigsResponse getErrorResponse(int throttleTimeMs, Throwable e) {
Errors error = Errors.forException(e); Errors error = Errors.forException(e);
@ -77,6 +68,6 @@ public class DescribeConfigsRequest extends AbstractRequest {
} }
public static DescribeConfigsRequest parse(ByteBuffer buffer, short version) { public static DescribeConfigsRequest parse(ByteBuffer buffer, short version) {
return new DescribeConfigsRequest(ApiKeys.DESCRIBE_CONFIGS.parseRequest(version, buffer), version); return new DescribeConfigsRequest(new DescribeConfigsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -19,10 +19,9 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.message.DescribeConfigsResponseData; import org.apache.kafka.common.message.DescribeConfigsResponseData;
import org.apache.kafka.common.message.DescribeConfigsResponseData.DescribeConfigsResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collection; import java.util.Collection;
@ -219,13 +218,16 @@ public class DescribeConfigsResponse extends AbstractResponse {
private final DescribeConfigsResponseData data; private final DescribeConfigsResponseData data;
public DescribeConfigsResponse(DescribeConfigsResponseData data) { public DescribeConfigsResponse(DescribeConfigsResponseData data) {
super(ApiKeys.DESCRIBE_CONFIGS);
this.data = data; this.data = data;
} }
public DescribeConfigsResponse(Struct struct, short version) { // This constructor should only be used after deserialization, it has special handling for version 0
this.data = new DescribeConfigsResponseData(struct, version); private DescribeConfigsResponse(DescribeConfigsResponseData data, short version) {
super(ApiKeys.DESCRIBE_CONFIGS);
this.data = data;
if (version == 0) { if (version == 0) {
for (DescribeConfigsResult result : data.results()) { for (DescribeConfigsResponseData.DescribeConfigsResult result : data.results()) {
for (DescribeConfigsResponseData.DescribeConfigsResourceResult config : result.configs()) { for (DescribeConfigsResponseData.DescribeConfigsResourceResult config : result.configs()) {
if (config.isDefault()) { if (config.isDefault()) {
config.setConfigSource(ConfigSource.DEFAULT_CONFIG.id); config.setConfigSource(ConfigSource.DEFAULT_CONFIG.id);
@ -243,6 +245,7 @@ public class DescribeConfigsResponse extends AbstractResponse {
} }
} }
@Override
public DescribeConfigsResponseData data() { public DescribeConfigsResponseData data() {
return data; return data;
} }
@ -261,13 +264,8 @@ public class DescribeConfigsResponse extends AbstractResponse {
return errorCounts; return errorCounts;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
public static DescribeConfigsResponse parse(ByteBuffer buffer, short version) { public static DescribeConfigsResponse parse(ByteBuffer buffer, short version) {
return new DescribeConfigsResponse(ApiKeys.DESCRIBE_CONFIGS.parseResponse(version, buffer), version); return new DescribeConfigsResponse(new DescribeConfigsResponseData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override

View File

@ -18,17 +18,16 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeDelegationTokenRequestData; import org.apache.kafka.common.message.DescribeDelegationTokenRequestData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.KafkaPrincipal;
import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class DescribeDelegationTokenRequest extends AbstractRequest { public class DescribeDelegationTokenRequest extends AbstractRequest {
private final DescribeDelegationTokenRequestData data;
public static class Builder extends AbstractRequest.Builder<DescribeDelegationTokenRequest> { public static class Builder extends AbstractRequest.Builder<DescribeDelegationTokenRequest> {
private final DescribeDelegationTokenRequestData data; private final DescribeDelegationTokenRequestData data;
@ -54,21 +53,13 @@ public class DescribeDelegationTokenRequest extends AbstractRequest {
} }
} }
public DescribeDelegationTokenRequest(Struct struct, short version) { private final DescribeDelegationTokenRequestData data;
super(ApiKeys.DESCRIBE_DELEGATION_TOKEN, version);
this.data = new DescribeDelegationTokenRequestData(struct, version);
}
public DescribeDelegationTokenRequest(DescribeDelegationTokenRequestData data, short version) { public DescribeDelegationTokenRequest(DescribeDelegationTokenRequestData data, short version) {
super(ApiKeys.DESCRIBE_DELEGATION_TOKEN, version); super(ApiKeys.DESCRIBE_DELEGATION_TOKEN, version);
this.data = data; this.data = data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
public DescribeDelegationTokenRequestData data() { public DescribeDelegationTokenRequestData data() {
return data; return data;
} }
@ -81,4 +72,9 @@ public class DescribeDelegationTokenRequest extends AbstractRequest {
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
return new DescribeDelegationTokenResponse(throttleTimeMs, Errors.forException(e)); return new DescribeDelegationTokenResponse(throttleTimeMs, Errors.forException(e));
} }
public static DescribeDelegationTokenRequest parse(ByteBuffer buffer, short version) {
return new DescribeDelegationTokenRequest(new DescribeDelegationTokenRequestData(
new ByteBufferAccessor(buffer), version), version);
}
} }

View File

@ -20,8 +20,8 @@ import org.apache.kafka.common.message.DescribeDelegationTokenResponseData;
import org.apache.kafka.common.message.DescribeDelegationTokenResponseData.DescribedDelegationToken; import org.apache.kafka.common.message.DescribeDelegationTokenResponseData.DescribedDelegationToken;
import org.apache.kafka.common.message.DescribeDelegationTokenResponseData.DescribedDelegationTokenRenewer; import org.apache.kafka.common.message.DescribeDelegationTokenResponseData.DescribedDelegationTokenRenewer;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.KafkaPrincipal;
import org.apache.kafka.common.security.token.delegation.DelegationToken; import org.apache.kafka.common.security.token.delegation.DelegationToken;
import org.apache.kafka.common.security.token.delegation.TokenInformation; import org.apache.kafka.common.security.token.delegation.TokenInformation;
@ -37,6 +37,7 @@ public class DescribeDelegationTokenResponse extends AbstractResponse {
private final DescribeDelegationTokenResponseData data; private final DescribeDelegationTokenResponseData data;
public DescribeDelegationTokenResponse(int throttleTimeMs, Errors error, List<DelegationToken> tokens) { public DescribeDelegationTokenResponse(int throttleTimeMs, Errors error, List<DelegationToken> tokens) {
super(ApiKeys.DESCRIBE_DELEGATION_TOKEN);
List<DescribedDelegationToken> describedDelegationTokenList = tokens List<DescribedDelegationToken> describedDelegationTokenList = tokens
.stream() .stream()
.map(dt -> new DescribedDelegationToken() .map(dt -> new DescribedDelegationToken()
@ -63,12 +64,14 @@ public class DescribeDelegationTokenResponse extends AbstractResponse {
this(throttleTimeMs, error, new ArrayList<>()); this(throttleTimeMs, error, new ArrayList<>());
} }
public DescribeDelegationTokenResponse(Struct struct, short version) { public DescribeDelegationTokenResponse(DescribeDelegationTokenResponseData data) {
this.data = new DescribeDelegationTokenResponseData(struct, version); super(ApiKeys.DESCRIBE_DELEGATION_TOKEN);
this.data = data;
} }
public static DescribeDelegationTokenResponse parse(ByteBuffer buffer, short version) { public static DescribeDelegationTokenResponse parse(ByteBuffer buffer, short version) {
return new DescribeDelegationTokenResponse(ApiKeys.DESCRIBE_DELEGATION_TOKEN.responseSchema(version).read(buffer), version); return new DescribeDelegationTokenResponse(new DescribeDelegationTokenResponseData(
new ByteBufferAccessor(buffer), version));
} }
@Override @Override
@ -77,8 +80,8 @@ public class DescribeDelegationTokenResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected DescribeDelegationTokenResponseData data() {
return data.toStruct(version); return data;
} }
@Override @Override

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeGroupsRequestData; import org.apache.kafka.common.message.DescribeGroupsRequestData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -46,32 +46,19 @@ public class DescribeGroupsRequest extends AbstractRequest {
} }
private final DescribeGroupsRequestData data; private final DescribeGroupsRequestData data;
private final short version;
private DescribeGroupsRequest(DescribeGroupsRequestData data, short version) { private DescribeGroupsRequest(DescribeGroupsRequestData data, short version) {
super(ApiKeys.DESCRIBE_GROUPS, version); super(ApiKeys.DESCRIBE_GROUPS, version);
this.data = data; this.data = data;
this.version = version;
}
public DescribeGroupsRequest(Struct struct, short version) {
super(ApiKeys.DESCRIBE_GROUPS, version);
this.data = new DescribeGroupsRequestData(struct, version);
this.version = version;
} }
public DescribeGroupsRequestData data() { public DescribeGroupsRequestData data() {
return data; return data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
if (version == 0) { if (version() == 0) {
return DescribeGroupsResponse.fromError(DEFAULT_THROTTLE_TIME, Errors.forException(e), data.groups()); return DescribeGroupsResponse.fromError(DEFAULT_THROTTLE_TIME, Errors.forException(e), data.groups());
} else { } else {
return DescribeGroupsResponse.fromError(throttleTimeMs, Errors.forException(e), data.groups()); return DescribeGroupsResponse.fromError(throttleTimeMs, Errors.forException(e), data.groups());
@ -79,6 +66,6 @@ public class DescribeGroupsRequest extends AbstractRequest {
} }
public static DescribeGroupsRequest parse(ByteBuffer buffer, short version) { public static DescribeGroupsRequest parse(ByteBuffer buffer, short version) {
return new DescribeGroupsRequest(ApiKeys.DESCRIBE_GROUPS.parseRequest(version, buffer), version); return new DescribeGroupsRequest(new DescribeGroupsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -20,8 +20,8 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData;
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup;
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -47,13 +47,10 @@ public class DescribeGroupsResponse extends AbstractResponse {
private DescribeGroupsResponseData data; private DescribeGroupsResponseData data;
public DescribeGroupsResponse(DescribeGroupsResponseData data) { public DescribeGroupsResponse(DescribeGroupsResponseData data) {
super(ApiKeys.DESCRIBE_GROUPS);
this.data = data; this.data = data;
} }
public DescribeGroupsResponse(Struct struct, short version) {
this.data = new DescribeGroupsResponseData(struct, version);
}
public static DescribedGroupMember groupMember( public static DescribedGroupMember groupMember(
final String memberId, final String memberId,
final String groupInstanceId, final String groupInstanceId,
@ -112,11 +109,6 @@ public class DescribeGroupsResponse extends AbstractResponse {
return data; return data;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -129,9 +121,8 @@ public class DescribeGroupsResponse extends AbstractResponse {
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errorCounts = new HashMap<>(); Map<Errors, Integer> errorCounts = new HashMap<>();
data.groups().forEach(describedGroup -> { data.groups().forEach(describedGroup ->
updateErrorCounts(errorCounts, Errors.forCode(describedGroup.errorCode())); updateErrorCounts(errorCounts, Errors.forCode(describedGroup.errorCode())));
});
return errorCounts; return errorCounts;
} }
@ -149,8 +140,7 @@ public class DescribeGroupsResponse extends AbstractResponse {
} }
public static DescribeGroupsResponse parse(ByteBuffer buffer, short version) { public static DescribeGroupsResponse parse(ByteBuffer buffer, short version) {
return new DescribeGroupsResponse( return new DescribeGroupsResponse(new DescribeGroupsResponseData(new ByteBufferAccessor(buffer), version));
ApiKeys.DESCRIBE_GROUPS.responseSchema(version).read(buffer), version);
} }
@Override @Override

View File

@ -20,7 +20,7 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeLogDirsRequestData; import org.apache.kafka.common.message.DescribeLogDirsRequestData;
import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.message.DescribeLogDirsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -47,25 +47,16 @@ public class DescribeLogDirsRequest extends AbstractRequest {
} }
} }
public DescribeLogDirsRequest(Struct struct, short version) {
super(ApiKeys.DESCRIBE_LOG_DIRS, version);
this.data = new DescribeLogDirsRequestData(struct, version);
}
public DescribeLogDirsRequest(DescribeLogDirsRequestData data, short version) { public DescribeLogDirsRequest(DescribeLogDirsRequestData data, short version) {
super(ApiKeys.DESCRIBE_LOG_DIRS, version); super(ApiKeys.DESCRIBE_LOG_DIRS, version);
this.data = data; this.data = data;
} }
@Override
public DescribeLogDirsRequestData data() { public DescribeLogDirsRequestData data() {
return data; return data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
return new DescribeLogDirsResponse(new DescribeLogDirsResponseData().setThrottleTimeMs(throttleTimeMs)); return new DescribeLogDirsResponse(new DescribeLogDirsResponseData().setThrottleTimeMs(throttleTimeMs));
@ -76,6 +67,6 @@ public class DescribeLogDirsRequest extends AbstractRequest {
} }
public static DescribeLogDirsRequest parse(ByteBuffer buffer, short version) { public static DescribeLogDirsRequest parse(ByteBuffer buffer, short version) {
return new DescribeLogDirsRequest(ApiKeys.DESCRIBE_LOG_DIRS.parseRequest(version, buffer), version); return new DescribeLogDirsRequest(new DescribeLogDirsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -20,8 +20,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.message.DescribeLogDirsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -34,11 +34,8 @@ public class DescribeLogDirsResponse extends AbstractResponse {
private final DescribeLogDirsResponseData data; private final DescribeLogDirsResponseData data;
public DescribeLogDirsResponse(Struct struct, short version) {
this.data = new DescribeLogDirsResponseData(struct, version);
}
public DescribeLogDirsResponse(DescribeLogDirsResponseData data) { public DescribeLogDirsResponse(DescribeLogDirsResponseData data) {
super(ApiKeys.DESCRIBE_LOG_DIRS);
this.data = data; this.data = data;
} }
@ -46,11 +43,6 @@ public class DescribeLogDirsResponse extends AbstractResponse {
return data; return data;
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -66,7 +58,7 @@ public class DescribeLogDirsResponse extends AbstractResponse {
} }
public static DescribeLogDirsResponse parse(ByteBuffer buffer, short version) { public static DescribeLogDirsResponse parse(ByteBuffer buffer, short version) {
return new DescribeLogDirsResponse(ApiKeys.DESCRIBE_LOG_DIRS.responseSchema(version).read(buffer), version); return new DescribeLogDirsResponse(new DescribeLogDirsResponseData(new ByteBufferAccessor(buffer), version));
} }
// Note this class is part of the public API, reachable from Admin.describeLogDirs() // Note this class is part of the public API, reachable from Admin.describeLogDirs()

View File

@ -20,9 +20,10 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.DescribeQuorumRequestData; import org.apache.kafka.common.message.DescribeQuorumRequestData;
import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.DescribeQuorumResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
@ -55,13 +56,8 @@ public class DescribeQuorumRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public DescribeQuorumRequest(Struct struct, short version) { public static DescribeQuorumRequest parse(ByteBuffer buffer, short version) {
super(ApiKeys.DESCRIBE_QUORUM, version); return new DescribeQuorumRequest(new DescribeQuorumRequestData(new ByteBufferAccessor(buffer), version), version);
this.data = new DescribeQuorumRequestData(struct, version);
}
public DescribeQuorumRequest(DescribeQuorumRequestData data) {
this(data, (short) (DescribeQuorumRequestData.SCHEMAS.length - 1));
} }
public static DescribeQuorumRequestData singletonRequest(TopicPartition topicPartition) { public static DescribeQuorumRequestData singletonRequest(TopicPartition topicPartition) {
@ -76,8 +72,8 @@ public class DescribeQuorumRequest extends AbstractRequest {
} }
@Override @Override
protected Struct toStruct() { protected DescribeQuorumRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override

View File

@ -20,8 +20,8 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.DescribeQuorumResponseData;
import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -44,18 +44,10 @@ public class DescribeQuorumResponse extends AbstractResponse {
public final DescribeQuorumResponseData data; public final DescribeQuorumResponseData data;
public DescribeQuorumResponse(DescribeQuorumResponseData data) { public DescribeQuorumResponse(DescribeQuorumResponseData data) {
super(ApiKeys.DESCRIBE_QUORUM);
this.data = data; this.data = data;
} }
public DescribeQuorumResponse(Struct struct, short version) {
this.data = new DescribeQuorumResponseData(struct, version);
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errors = new HashMap<>(); Map<Errors, Integer> errors = new HashMap<>();
@ -70,6 +62,16 @@ public class DescribeQuorumResponse extends AbstractResponse {
return errors; return errors;
} }
@Override
protected DescribeQuorumResponseData data() {
return data;
}
@Override
public int throttleTimeMs() {
return DEFAULT_THROTTLE_TIME;
}
public static DescribeQuorumResponseData singletonResponse(TopicPartition topicPartition, public static DescribeQuorumResponseData singletonResponse(TopicPartition topicPartition,
int leaderId, int leaderId,
int leaderEpoch, int leaderEpoch,
@ -89,6 +91,6 @@ public class DescribeQuorumResponse extends AbstractResponse {
} }
public static DescribeQuorumResponse parse(ByteBuffer buffer, short version) { public static DescribeQuorumResponse parse(ByteBuffer buffer, short version) {
return new DescribeQuorumResponse(ApiKeys.DESCRIBE_QUORUM.responseSchema(version).read(buffer), version); return new DescribeQuorumResponse(new DescribeQuorumResponseData(new ByteBufferAccessor(buffer), version));
} }
} }

View File

@ -19,7 +19,7 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData;
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -53,30 +53,15 @@ public class DescribeUserScramCredentialsRequest extends AbstractRequest {
this.version = version; this.version = version;
} }
DescribeUserScramCredentialsRequest(Struct struct, short version) {
super(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS, version);
this.data = new DescribeUserScramCredentialsRequestData(struct, version);
this.version = version;
}
public static DescribeUserScramCredentialsRequest parse(ByteBuffer buffer, short version) { public static DescribeUserScramCredentialsRequest parse(ByteBuffer buffer, short version) {
return new DescribeUserScramCredentialsRequest( return new DescribeUserScramCredentialsRequest(new DescribeUserScramCredentialsRequestData(
ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.parseRequest(version, buffer), version new ByteBufferAccessor(buffer), version), version);
);
} }
public DescribeUserScramCredentialsRequestData data() { public DescribeUserScramCredentialsRequestData data() {
return data; return data;
} }
/**
* Visible for testing.
*/
@Override
public Struct toStruct() {
return data.toStruct(version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
ApiError apiError = ApiError.fromThrowable(e); ApiError apiError = ApiError.fromThrowable(e);

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -28,22 +28,11 @@ public class DescribeUserScramCredentialsResponse extends AbstractResponse {
private final DescribeUserScramCredentialsResponseData data; private final DescribeUserScramCredentialsResponseData data;
public DescribeUserScramCredentialsResponse(Struct struct) {
this(struct, ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.latestVersion());
}
public DescribeUserScramCredentialsResponse(DescribeUserScramCredentialsResponseData responseData) { public DescribeUserScramCredentialsResponse(DescribeUserScramCredentialsResponseData responseData) {
super(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS);
this.data = responseData; this.data = responseData;
} }
DescribeUserScramCredentialsResponse(Struct struct, short version) {
this.data = new DescribeUserScramCredentialsResponseData(struct, version);
}
public static DescribeUserScramCredentialsResponse parse(ByteBuffer buffer, short version) {
return new DescribeUserScramCredentialsResponse(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.responseSchema(version).read(buffer), version);
}
public DescribeUserScramCredentialsResponseData data() { public DescribeUserScramCredentialsResponseData data() {
return data; return data;
} }
@ -63,8 +52,7 @@ public class DescribeUserScramCredentialsResponse extends AbstractResponse {
return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode()))); return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode())));
} }
@Override public static DescribeUserScramCredentialsResponse parse(ByteBuffer buffer, short version) {
protected Struct toStruct(short version) { return new DescribeUserScramCredentialsResponse(new DescribeUserScramCredentialsResponseData(new ByteBufferAccessor(buffer), version));
return data.toStruct(version);
} }
} }

View File

@ -30,8 +30,8 @@ import org.apache.kafka.common.message.ElectLeadersRequestData;
import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.CollectionUtils;
public class ElectLeadersRequest extends AbstractRequest { public class ElectLeadersRequest extends AbstractRequest {
@ -90,11 +90,6 @@ public class ElectLeadersRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public ElectLeadersRequest(Struct struct, short version) {
super(ApiKeys.ELECT_LEADERS, version);
this.data = new ElectLeadersRequestData(struct, version);
}
public ElectLeadersRequestData data() { public ElectLeadersRequestData data() {
return data; return data;
} }
@ -124,11 +119,6 @@ public class ElectLeadersRequest extends AbstractRequest {
} }
public static ElectLeadersRequest parse(ByteBuffer buffer, short version) { public static ElectLeadersRequest parse(ByteBuffer buffer, short version) {
return new ElectLeadersRequest(ApiKeys.ELECT_LEADERS.parseRequest(version, buffer), version); return new ElectLeadersRequest(new ElectLeadersRequestData(new ByteBufferAccessor(buffer), version), version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version());
} }
} }

View File

@ -25,30 +25,18 @@ import java.util.Optional;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.ElectLeadersResponseData;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
public class ElectLeadersResponse extends AbstractResponse { public class ElectLeadersResponse extends AbstractResponse {
private final short version;
private final ElectLeadersResponseData data; private final ElectLeadersResponseData data;
public ElectLeadersResponse(Struct struct) { public ElectLeadersResponse(ElectLeadersResponseData data) {
this(struct, ApiKeys.ELECT_LEADERS.latestVersion()); super(ApiKeys.ELECT_LEADERS);
} this.data = data;
public ElectLeadersResponse(Struct struct, short version) {
this.version = version;
this.data = new ElectLeadersResponseData(struct, version);
}
public ElectLeadersResponse(
int throttleTimeMs,
short errorCode,
List<ReplicaElectionResult> electionResults) {
this(throttleTimeMs, errorCode, electionResults, ApiKeys.ELECT_LEADERS.latestVersion());
} }
public ElectLeadersResponse( public ElectLeadersResponse(
@ -56,16 +44,11 @@ public class ElectLeadersResponse extends AbstractResponse {
short errorCode, short errorCode,
List<ReplicaElectionResult> electionResults, List<ReplicaElectionResult> electionResults,
short version) { short version) {
super(ApiKeys.ELECT_LEADERS);
this.version = version;
this.data = new ElectLeadersResponseData(); this.data = new ElectLeadersResponseData();
data.setThrottleTimeMs(throttleTimeMs); data.setThrottleTimeMs(throttleTimeMs);
if (version >= 1)
if (version >= 1) {
data.setErrorCode(errorCode); data.setErrorCode(errorCode);
}
data.setReplicaElectionResults(electionResults); data.setReplicaElectionResults(electionResults);
} }
@ -73,15 +56,6 @@ public class ElectLeadersResponse extends AbstractResponse {
return data; return data;
} }
public short version() {
return version;
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -100,7 +74,7 @@ public class ElectLeadersResponse extends AbstractResponse {
} }
public static ElectLeadersResponse parse(ByteBuffer buffer, short version) { public static ElectLeadersResponse parse(ByteBuffer buffer, short version) {
return new ElectLeadersResponse(ApiKeys.ELECT_LEADERS.responseSchema(version).read(buffer), version); return new ElectLeadersResponse(new ElectLeadersResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -20,9 +20,10 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochRequestData;
import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
@ -53,14 +54,9 @@ public class EndQuorumEpochRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public EndQuorumEpochRequest(Struct struct, short version) {
super(ApiKeys.END_QUORUM_EPOCH, version);
this.data = new EndQuorumEpochRequestData(struct, version);
}
@Override @Override
protected Struct toStruct() { protected EndQuorumEpochRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -69,6 +65,10 @@ public class EndQuorumEpochRequest extends AbstractRequest {
.setErrorCode(Errors.forException(e).code())); .setErrorCode(Errors.forException(e).code()));
} }
public static EndQuorumEpochRequest parse(ByteBuffer buffer, short version) {
return new EndQuorumEpochRequest(new EndQuorumEpochRequestData(new ByteBufferAccessor(buffer), version), version);
}
public static EndQuorumEpochRequestData singletonRequest(TopicPartition topicPartition, public static EndQuorumEpochRequestData singletonRequest(TopicPartition topicPartition,
int leaderEpoch, int leaderEpoch,
int leaderId, int leaderId,

View File

@ -20,8 +20,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -45,23 +45,10 @@ public class EndQuorumEpochResponse extends AbstractResponse {
public final EndQuorumEpochResponseData data; public final EndQuorumEpochResponseData data;
public EndQuorumEpochResponse(EndQuorumEpochResponseData data) { public EndQuorumEpochResponse(EndQuorumEpochResponseData data) {
super(ApiKeys.END_QUORUM_EPOCH);
this.data = data; this.data = data;
} }
public EndQuorumEpochResponse(Struct struct, short version) {
this.data = new EndQuorumEpochResponseData(struct, version);
}
public EndQuorumEpochResponse(Struct struct) {
short latestVersion = (short) (EndQuorumEpochResponseData.SCHEMAS.length - 1);
this.data = new EndQuorumEpochResponseData(struct, latestVersion);
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
@Override @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errors = new HashMap<>(); Map<Errors, Integer> errors = new HashMap<>();
@ -76,6 +63,16 @@ public class EndQuorumEpochResponse extends AbstractResponse {
return errors; return errors;
} }
@Override
protected EndQuorumEpochResponseData data() {
return data;
}
@Override
public int throttleTimeMs() {
return DEFAULT_THROTTLE_TIME;
}
public static EndQuorumEpochResponseData singletonResponse( public static EndQuorumEpochResponseData singletonResponse(
Errors topLevelError, Errors topLevelError,
TopicPartition topicPartition, TopicPartition topicPartition,
@ -98,6 +95,6 @@ public class EndQuorumEpochResponse extends AbstractResponse {
} }
public static EndQuorumEpochResponse parse(ByteBuffer buffer, short version) { public static EndQuorumEpochResponse parse(ByteBuffer buffer, short version) {
return new EndQuorumEpochResponse(ApiKeys.END_QUORUM_EPOCH.responseSchema(version).read(buffer), version); return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(new ByteBufferAccessor(buffer), version));
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.EndTxnRequestData; import org.apache.kafka.common.message.EndTxnRequestData;
import org.apache.kafka.common.message.EndTxnResponseData; import org.apache.kafka.common.message.EndTxnResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -52,11 +52,6 @@ public class EndTxnRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public EndTxnRequest(Struct struct, short version) {
super(ApiKeys.END_TXN, version);
this.data = new EndTxnRequestData(struct, version);
}
public TransactionResult result() { public TransactionResult result() {
if (data.committed()) if (data.committed())
return TransactionResult.COMMIT; return TransactionResult.COMMIT;
@ -65,8 +60,8 @@ public class EndTxnRequest extends AbstractRequest {
} }
@Override @Override
protected Struct toStruct() { protected EndTxnRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override
@ -78,6 +73,6 @@ public class EndTxnRequest extends AbstractRequest {
} }
public static EndTxnRequest parse(ByteBuffer buffer, short version) { public static EndTxnRequest parse(ByteBuffer buffer, short version) {
return new EndTxnRequest(ApiKeys.END_TXN.parseRequest(version, buffer), version); return new EndTxnRequest(new EndTxnRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.EndTxnResponseData; import org.apache.kafka.common.message.EndTxnResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -41,18 +41,10 @@ public class EndTxnResponse extends AbstractResponse {
public final EndTxnResponseData data; public final EndTxnResponseData data;
public EndTxnResponse(EndTxnResponseData data) { public EndTxnResponse(EndTxnResponseData data) {
super(ApiKeys.END_TXN);
this.data = data; this.data = data;
} }
public EndTxnResponse(Struct struct) {
this(struct, (short) (EndTxnResponseData.SCHEMAS.length - 1));
}
public EndTxnResponse(Struct struct, short version) {
this.data = new EndTxnResponseData(struct, version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -69,12 +61,12 @@ public class EndTxnResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected EndTxnResponseData data() {
return data.toStruct(version); return data;
} }
public static EndTxnResponse parse(ByteBuffer buffer, short version) { public static EndTxnResponse parse(ByteBuffer buffer, short version) {
return new EndTxnResponse(ApiKeys.END_TXN.parseResponse(version, buffer), version); return new EndTxnResponse(new EndTxnResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -18,11 +18,9 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.EnvelopeRequestData; import org.apache.kafka.common.message.EnvelopeRequestData;
import org.apache.kafka.common.message.EnvelopeResponseData; import org.apache.kafka.common.message.EnvelopeResponseData;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.SendBuilder;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -60,11 +58,6 @@ public class EnvelopeRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public EnvelopeRequest(Struct struct, short version) {
super(ApiKeys.ENVELOPE, version);
this.data = new EnvelopeRequestData(struct, version);
}
public ByteBuffer requestData() { public ByteBuffer requestData() {
return data.requestData(); return data.requestData();
} }
@ -77,11 +70,6 @@ public class EnvelopeRequest extends AbstractRequest {
return data.requestPrincipal(); return data.requestPrincipal();
} }
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
return new EnvelopeResponse(new EnvelopeResponseData() return new EnvelopeResponse(new EnvelopeResponseData()
@ -89,16 +77,10 @@ public class EnvelopeRequest extends AbstractRequest {
} }
public static EnvelopeRequest parse(ByteBuffer buffer, short version) { public static EnvelopeRequest parse(ByteBuffer buffer, short version) {
return new EnvelopeRequest(ApiKeys.ENVELOPE.parseRequest(version, buffer), version); return new EnvelopeRequest(new EnvelopeRequestData(new ByteBufferAccessor(buffer), version), version);
} }
public EnvelopeRequestData data() { public EnvelopeRequestData data() {
return data; return data;
} }
@Override
public Send toSend(String destination, RequestHeader header) {
return SendBuilder.buildRequestSend(destination, header, this.data);
}
} }

View File

@ -17,10 +17,9 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.EnvelopeResponseData; import org.apache.kafka.common.message.EnvelopeResponseData;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.SendBuilder; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -29,8 +28,8 @@ public class EnvelopeResponse extends AbstractResponse {
private final EnvelopeResponseData data; private final EnvelopeResponseData data;
public EnvelopeResponse(ByteBuffer responseData, public EnvelopeResponse(ByteBuffer responseData, Errors error) {
Errors error) { super(ApiKeys.ENVELOPE);
this.data = new EnvelopeResponseData() this.data = new EnvelopeResponseData()
.setResponseData(responseData) .setResponseData(responseData)
.setErrorCode(error.code()); .setErrorCode(error.code());
@ -41,6 +40,7 @@ public class EnvelopeResponse extends AbstractResponse {
} }
public EnvelopeResponse(EnvelopeResponseData data) { public EnvelopeResponse(EnvelopeResponseData data) {
super(ApiKeys.ENVELOPE);
this.data = data; this.data = data;
} }
@ -53,11 +53,6 @@ public class EnvelopeResponse extends AbstractResponse {
return errorCounts(error()); return errorCounts(error());
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
public Errors error() { public Errors error() {
return Errors.forCode(data.errorCode()); return Errors.forCode(data.errorCode());
} }
@ -67,8 +62,12 @@ public class EnvelopeResponse extends AbstractResponse {
} }
@Override @Override
protected Send toSend(String destination, ResponseHeader header, short apiVersion) { public int throttleTimeMs() {
return SendBuilder.buildResponseSend(destination, header, this.data, apiVersion); return DEFAULT_THROTTLE_TIME;
}
public static EnvelopeResponse parse(ByteBuffer buffer, short version) {
return new EnvelopeResponse(new EnvelopeResponseData(new ByteBufferAccessor(buffer), version));
} }
} }

View File

@ -21,8 +21,8 @@ import java.nio.ByteBuffer;
import org.apache.kafka.common.message.ExpireDelegationTokenRequestData; import org.apache.kafka.common.message.ExpireDelegationTokenRequestData;
import org.apache.kafka.common.message.ExpireDelegationTokenResponseData; import org.apache.kafka.common.message.ExpireDelegationTokenResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
public class ExpireDelegationTokenRequest extends AbstractRequest { public class ExpireDelegationTokenRequest extends AbstractRequest {
@ -33,18 +33,14 @@ public class ExpireDelegationTokenRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public ExpireDelegationTokenRequest(Struct struct, short version) {
super(ApiKeys.EXPIRE_DELEGATION_TOKEN, version);
this.data = new ExpireDelegationTokenRequestData(struct, version);
}
public static ExpireDelegationTokenRequest parse(ByteBuffer buffer, short version) { public static ExpireDelegationTokenRequest parse(ByteBuffer buffer, short version) {
return new ExpireDelegationTokenRequest(ApiKeys.EXPIRE_DELEGATION_TOKEN.parseRequest(version, buffer), version); return new ExpireDelegationTokenRequest(
new ExpireDelegationTokenRequestData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override
protected Struct toStruct() { protected ExpireDelegationTokenRequestData data() {
return data.toStruct(version()); return data;
} }
@Override @Override

View File

@ -21,23 +21,21 @@ import java.util.Map;
import org.apache.kafka.common.message.ExpireDelegationTokenResponseData; import org.apache.kafka.common.message.ExpireDelegationTokenResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
public class ExpireDelegationTokenResponse extends AbstractResponse { public class ExpireDelegationTokenResponse extends AbstractResponse {
private final ExpireDelegationTokenResponseData data; private final ExpireDelegationTokenResponseData data;
public ExpireDelegationTokenResponse(ExpireDelegationTokenResponseData data) { public ExpireDelegationTokenResponse(ExpireDelegationTokenResponseData data) {
super(ApiKeys.EXPIRE_DELEGATION_TOKEN);
this.data = data; this.data = data;
} }
public ExpireDelegationTokenResponse(Struct struct, short version) {
this.data = new ExpireDelegationTokenResponseData(struct, version);
}
public static ExpireDelegationTokenResponse parse(ByteBuffer buffer, short version) { public static ExpireDelegationTokenResponse parse(ByteBuffer buffer, short version) {
return new ExpireDelegationTokenResponse(ApiKeys.EXPIRE_DELEGATION_TOKEN.responseSchema(version).read(buffer), version); return new ExpireDelegationTokenResponse(new ExpireDelegationTokenResponseData(new ByteBufferAccessor(buffer),
version));
} }
public Errors error() { public Errors error() {
@ -54,8 +52,8 @@ public class ExpireDelegationTokenResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected ExpireDelegationTokenResponseData data() {
return data.toStruct(version); return data;
} }
@Override @Override

View File

@ -19,12 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.RequestHeaderData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -55,10 +51,6 @@ public class FetchRequest extends AbstractRequest {
private final List<TopicPartition> toForget; private final List<TopicPartition> toForget;
private final FetchMetadata metadata; private final FetchMetadata metadata;
public FetchRequestData data() {
return data;
}
public static final class PartitionData { public static final class PartitionData {
public final long fetchOffset; public final long fetchOffset;
public final long logStartOffset; public final long logStartOffset;
@ -378,34 +370,12 @@ public class FetchRequest extends AbstractRequest {
return data.rackId(); return data.rackId();
} }
@Override
public ByteBuffer serialize(RequestHeader header) {
// Unlike the custom FetchResponse#toSend, we don't include the buffer size here. This buffer is passed
// to a NetworkSend which adds the length value in the eventual serialization
ObjectSerializationCache cache = new ObjectSerializationCache();
RequestHeaderData requestHeaderData = header.data();
int headerSize = requestHeaderData.size(cache, header.headerVersion());
int bodySize = data.size(cache, header.apiVersion());
ByteBuffer buffer = ByteBuffer.allocate(headerSize + bodySize);
ByteBufferAccessor writer = new ByteBufferAccessor(buffer);
requestHeaderData.write(writer, cache, header.headerVersion());
data.write(writer, cache, header.apiVersion());
buffer.rewind();
return buffer;
}
// For testing
public static FetchRequest parse(ByteBuffer buffer, short version) { public static FetchRequest parse(ByteBuffer buffer, short version) {
return new FetchRequest(new FetchRequestData(ApiKeys.FETCH.parseRequest(version, buffer), version), version); return new FetchRequest(new FetchRequestData(ApiKeys.FETCH.parseRequest(version, buffer), version), version);
} }
@Override @Override
protected Struct toStruct() { public FetchRequestData data() {
return data.toStruct(version()); return data;
} }
} }

View File

@ -18,12 +18,10 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.protocol.SendBuilder;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.BaseRecords; import org.apache.kafka.common.record.BaseRecords;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
@ -273,25 +271,17 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
LinkedHashMap<TopicPartition, PartitionData<T>> responseData, LinkedHashMap<TopicPartition, PartitionData<T>> responseData,
int throttleTimeMs, int throttleTimeMs,
int sessionId) { int sessionId) {
super(ApiKeys.FETCH);
this.data = toMessage(throttleTimeMs, error, responseData.entrySet().iterator(), sessionId); this.data = toMessage(throttleTimeMs, error, responseData.entrySet().iterator(), sessionId);
this.responseDataMap = responseData; this.responseDataMap = responseData;
} }
public FetchResponse(FetchResponseData fetchResponseData) { public FetchResponse(FetchResponseData fetchResponseData) {
super(ApiKeys.FETCH);
this.data = fetchResponseData; this.data = fetchResponseData;
this.responseDataMap = toResponseDataMap(fetchResponseData); this.responseDataMap = toResponseDataMap(fetchResponseData);
} }
@Override
public Struct toStruct(short version) {
return data.toStruct(version);
}
@Override
public Send toSend(String dest, ResponseHeader responseHeader, short apiVersion) {
return SendBuilder.buildResponseSend(dest, responseHeader, this.data, apiVersion);
}
public Errors error() { public Errors error() {
return Errors.forCode(data.errorCode()); return Errors.forCode(data.errorCode());
} }
@ -320,10 +310,7 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
} }
public static FetchResponse<MemoryRecords> parse(ByteBuffer buffer, short version) { public static FetchResponse<MemoryRecords> parse(ByteBuffer buffer, short version) {
FetchResponseData fetchResponseData = new FetchResponseData(); return new FetchResponse<>(new FetchResponseData(new ByteBufferAccessor(buffer), version));
ByteBufferAccessor reader = new ByteBufferAccessor(buffer);
fetchResponseData.read(reader, version);
return new FetchResponse<>(fetchResponseData);
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")

View File

@ -21,12 +21,11 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.message.FindCoordinatorRequestData;
import org.apache.kafka.common.message.FindCoordinatorResponseData; import org.apache.kafka.common.message.FindCoordinatorResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class FindCoordinatorRequest extends AbstractRequest { public class FindCoordinatorRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<FindCoordinatorRequest> { public static class Builder extends AbstractRequest.Builder<FindCoordinatorRequest> {
@ -63,11 +62,6 @@ public class FindCoordinatorRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public FindCoordinatorRequest(Struct struct, short version) {
super(ApiKeys.FIND_COORDINATOR, version);
this.data = new FindCoordinatorRequestData(struct, version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
FindCoordinatorResponseData response = new FindCoordinatorResponseData(); FindCoordinatorResponseData response = new FindCoordinatorResponseData();
@ -79,12 +73,8 @@ public class FindCoordinatorRequest extends AbstractRequest {
} }
public static FindCoordinatorRequest parse(ByteBuffer buffer, short version) { public static FindCoordinatorRequest parse(ByteBuffer buffer, short version) {
return new FindCoordinatorRequest(ApiKeys.FIND_COORDINATOR.parseRequest(version, buffer), version); return new FindCoordinatorRequest(new FindCoordinatorRequestData(new ByteBufferAccessor(buffer), version),
} version);
@Override
protected Struct toStruct() {
return data.toStruct(version());
} }
public FindCoordinatorRequestData data() { public FindCoordinatorRequestData data() {

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.message.FindCoordinatorResponseData; import org.apache.kafka.common.message.FindCoordinatorResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -41,13 +41,10 @@ public class FindCoordinatorResponse extends AbstractResponse {
private final FindCoordinatorResponseData data; private final FindCoordinatorResponseData data;
public FindCoordinatorResponse(FindCoordinatorResponseData data) { public FindCoordinatorResponse(FindCoordinatorResponseData data) {
super(ApiKeys.FIND_COORDINATOR);
this.data = data; this.data = data;
} }
public FindCoordinatorResponse(Struct struct, short version) {
this.data = new FindCoordinatorResponseData(struct, version);
}
public FindCoordinatorResponseData data() { public FindCoordinatorResponseData data() {
return data; return data;
} }
@ -74,13 +71,8 @@ public class FindCoordinatorResponse extends AbstractResponse {
return errorCounts(error()); return errorCounts(error());
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
public static FindCoordinatorResponse parse(ByteBuffer buffer, short version) { public static FindCoordinatorResponse parse(ByteBuffer buffer, short version) {
return new FindCoordinatorResponse(ApiKeys.FIND_COORDINATOR.responseSchema(version).read(buffer), version); return new FindCoordinatorResponse(new FindCoordinatorResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -20,12 +20,11 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.HeartbeatRequestData; import org.apache.kafka.common.message.HeartbeatRequestData;
import org.apache.kafka.common.message.HeartbeatResponseData; import org.apache.kafka.common.message.HeartbeatResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class HeartbeatRequest extends AbstractRequest { public class HeartbeatRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<HeartbeatRequest> { public static class Builder extends AbstractRequest.Builder<HeartbeatRequest> {
@ -58,11 +57,6 @@ public class HeartbeatRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public HeartbeatRequest(Struct struct, short version) {
super(ApiKeys.HEARTBEAT, version);
this.data = new HeartbeatRequestData(struct, version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
HeartbeatResponseData responseData = new HeartbeatResponseData(). HeartbeatResponseData responseData = new HeartbeatResponseData().
@ -74,11 +68,11 @@ public class HeartbeatRequest extends AbstractRequest {
} }
public static HeartbeatRequest parse(ByteBuffer buffer, short version) { public static HeartbeatRequest parse(ByteBuffer buffer, short version) {
return new HeartbeatRequest(ApiKeys.HEARTBEAT.parseRequest(version, buffer), version); return new HeartbeatRequest(new HeartbeatRequestData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override
protected Struct toStruct() { protected HeartbeatRequestData data() {
return data.toStruct(version()); return data;
} }
} }

View File

@ -18,13 +18,12 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.HeartbeatResponseData; import org.apache.kafka.common.message.HeartbeatResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
public class HeartbeatResponse extends AbstractResponse { public class HeartbeatResponse extends AbstractResponse {
/** /**
@ -40,13 +39,10 @@ public class HeartbeatResponse extends AbstractResponse {
private final HeartbeatResponseData data; private final HeartbeatResponseData data;
public HeartbeatResponse(HeartbeatResponseData data) { public HeartbeatResponse(HeartbeatResponseData data) {
super(ApiKeys.HEARTBEAT);
this.data = data; this.data = data;
} }
public HeartbeatResponse(Struct struct, short version) {
this.data = new HeartbeatResponseData(struct, version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -62,12 +58,12 @@ public class HeartbeatResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected HeartbeatResponseData data() {
return data.toStruct(version); return data;
} }
public static HeartbeatResponse parse(ByteBuffer buffer, short version) { public static HeartbeatResponse parse(ByteBuffer buffer, short version) {
return new HeartbeatResponse(ApiKeys.HEARTBEAT.parseResponse(version, buffer), version); return new HeartbeatResponse(new HeartbeatResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -24,7 +24,7 @@ import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.AlterC
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData;
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collection; import java.util.Collection;
@ -86,25 +86,15 @@ public class IncrementalAlterConfigsRequest extends AbstractRequest {
this.version = version; this.version = version;
} }
IncrementalAlterConfigsRequest(final Struct struct, final short version) {
super(ApiKeys.INCREMENTAL_ALTER_CONFIGS, version);
this.data = new IncrementalAlterConfigsRequestData(struct, version);
this.version = version;
}
public static IncrementalAlterConfigsRequest parse(ByteBuffer buffer, short version) { public static IncrementalAlterConfigsRequest parse(ByteBuffer buffer, short version) {
return new IncrementalAlterConfigsRequest(ApiKeys.INCREMENTAL_ALTER_CONFIGS.parseRequest(version, buffer), version); return new IncrementalAlterConfigsRequest(new IncrementalAlterConfigsRequestData(
new ByteBufferAccessor(buffer), version), version);
} }
public IncrementalAlterConfigsRequestData data() { public IncrementalAlterConfigsRequestData data() {
return data; return data;
} }
@Override
protected Struct toStruct() {
return data.toStruct(version);
}
@Override @Override
public AbstractResponse getErrorResponse(final int throttleTimeMs, final Throwable e) { public AbstractResponse getErrorResponse(final int throttleTimeMs, final Throwable e) {
IncrementalAlterConfigsResponseData response = new IncrementalAlterConfigsResponseData(); IncrementalAlterConfigsResponseData response = new IncrementalAlterConfigsResponseData();

View File

@ -21,8 +21,8 @@ import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData;
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -34,6 +34,7 @@ public class IncrementalAlterConfigsResponse extends AbstractResponse {
public IncrementalAlterConfigsResponse(final int requestThrottleMs, public IncrementalAlterConfigsResponse(final int requestThrottleMs,
final Map<ConfigResource, ApiError> results) { final Map<ConfigResource, ApiError> results) {
super(ApiKeys.INCREMENTAL_ALTER_CONFIGS);
final List<AlterConfigsResourceResponse> newResults = new ArrayList<>(results.size()); final List<AlterConfigsResourceResponse> newResults = new ArrayList<>(results.size());
results.forEach( results.forEach(
(resource, error) -> newResults.add( (resource, error) -> newResults.add(
@ -61,13 +62,10 @@ public class IncrementalAlterConfigsResponse extends AbstractResponse {
private final IncrementalAlterConfigsResponseData data; private final IncrementalAlterConfigsResponseData data;
public IncrementalAlterConfigsResponse(IncrementalAlterConfigsResponseData data) { public IncrementalAlterConfigsResponse(IncrementalAlterConfigsResponseData data) {
super(ApiKeys.INCREMENTAL_ALTER_CONFIGS);
this.data = data; this.data = data;
} }
public IncrementalAlterConfigsResponse(final Struct struct, final short version) {
this.data = new IncrementalAlterConfigsResponseData(struct, version);
}
public IncrementalAlterConfigsResponseData data() { public IncrementalAlterConfigsResponseData data() {
return data; return data;
} }
@ -81,11 +79,6 @@ public class IncrementalAlterConfigsResponse extends AbstractResponse {
return counts; return counts;
} }
@Override
protected Struct toStruct(final short version) {
return data.toStruct(version);
}
@Override @Override
public boolean shouldClientThrottle(short version) { public boolean shouldClientThrottle(short version) {
return version >= 0; return version >= 0;
@ -97,7 +90,7 @@ public class IncrementalAlterConfigsResponse extends AbstractResponse {
} }
public static IncrementalAlterConfigsResponse parse(ByteBuffer buffer, short version) { public static IncrementalAlterConfigsResponse parse(ByteBuffer buffer, short version) {
return new IncrementalAlterConfigsResponse( return new IncrementalAlterConfigsResponse(new IncrementalAlterConfigsResponseData(
ApiKeys.INCREMENTAL_ALTER_CONFIGS.responseSchema(version).read(buffer), version); new ByteBufferAccessor(buffer), version));
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.InitProducerIdRequestData; import org.apache.kafka.common.message.InitProducerIdRequestData;
import org.apache.kafka.common.message.InitProducerIdResponseData; import org.apache.kafka.common.message.InitProducerIdResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -58,12 +58,6 @@ public class InitProducerIdRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public InitProducerIdRequest(Struct struct, short version) {
super(ApiKeys.INIT_PRODUCER_ID, version);
this.data = new InitProducerIdRequestData(struct, version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
InitProducerIdResponseData response = new InitProducerIdResponseData() InitProducerIdResponseData response = new InitProducerIdResponseData()
@ -75,12 +69,11 @@ public class InitProducerIdRequest extends AbstractRequest {
} }
public static InitProducerIdRequest parse(ByteBuffer buffer, short version) { public static InitProducerIdRequest parse(ByteBuffer buffer, short version) {
return new InitProducerIdRequest(ApiKeys.INIT_PRODUCER_ID.parseRequest(version, buffer), version); return new InitProducerIdRequest(new InitProducerIdRequestData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override
protected Struct toStruct() { protected InitProducerIdRequestData data() {
return data.toStruct(version()); return data;
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.InitProducerIdResponseData; import org.apache.kafka.common.message.InitProducerIdResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -39,13 +39,10 @@ public class InitProducerIdResponse extends AbstractResponse {
public final InitProducerIdResponseData data; public final InitProducerIdResponseData data;
public InitProducerIdResponse(InitProducerIdResponseData data) { public InitProducerIdResponse(InitProducerIdResponseData data) {
super(ApiKeys.INIT_PRODUCER_ID);
this.data = data; this.data = data;
} }
public InitProducerIdResponse(Struct struct, short version) {
this.data = new InitProducerIdResponseData(struct, version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -57,12 +54,12 @@ public class InitProducerIdResponse extends AbstractResponse {
} }
@Override @Override
protected Struct toStruct(short version) { protected InitProducerIdResponseData data() {
return data.toStruct(version); return data;
} }
public static InitProducerIdResponse parse(ByteBuffer buffer, short version) { public static InitProducerIdResponse parse(ByteBuffer buffer, short version) {
return new InitProducerIdResponse(ApiKeys.INIT_PRODUCER_ID.parseResponse(version, buffer), version); return new InitProducerIdResponse(new InitProducerIdResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -21,8 +21,8 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.JoinGroupResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -99,12 +99,6 @@ public class JoinGroupRequest extends AbstractRequest {
maybeOverrideRebalanceTimeout(version); maybeOverrideRebalanceTimeout(version);
} }
public JoinGroupRequest(Struct struct, short version) {
super(ApiKeys.JOIN_GROUP, version);
this.data = new JoinGroupRequestData(struct, version);
maybeOverrideRebalanceTimeout(version);
}
private void maybeOverrideRebalanceTimeout(short version) { private void maybeOverrideRebalanceTimeout(short version) {
if (version == 0) { if (version == 0) {
// Version 0 has no rebalance timeout, so we use the session timeout // Version 0 has no rebalance timeout, so we use the session timeout
@ -137,11 +131,6 @@ public class JoinGroupRequest extends AbstractRequest {
} }
public static JoinGroupRequest parse(ByteBuffer buffer, short version) { public static JoinGroupRequest parse(ByteBuffer buffer, short version) {
return new JoinGroupRequest(ApiKeys.JOIN_GROUP.parseRequest(version, buffer), version); return new JoinGroupRequest(new JoinGroupRequestData(new ByteBufferAccessor(buffer), version), version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version());
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.JoinGroupResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -29,18 +29,10 @@ public class JoinGroupResponse extends AbstractResponse {
private final JoinGroupResponseData data; private final JoinGroupResponseData data;
public JoinGroupResponse(JoinGroupResponseData data) { public JoinGroupResponse(JoinGroupResponseData data) {
super(ApiKeys.JOIN_GROUP);
this.data = data; this.data = data;
} }
public JoinGroupResponse(Struct struct) {
short latestVersion = (short) (JoinGroupResponseData.SCHEMAS.length - 1);
this.data = new JoinGroupResponseData(struct, latestVersion);
}
public JoinGroupResponse(Struct struct, short version) {
this.data = new JoinGroupResponseData(struct, version);
}
public JoinGroupResponseData data() { public JoinGroupResponseData data() {
return data; return data;
} }
@ -63,13 +55,8 @@ public class JoinGroupResponse extends AbstractResponse {
return errorCounts(Errors.forCode(data.errorCode())); return errorCounts(Errors.forCode(data.errorCode()));
} }
public static JoinGroupResponse parse(ByteBuffer buffer, short versionId) { public static JoinGroupResponse parse(ByteBuffer buffer, short version) {
return new JoinGroupResponse(ApiKeys.JOIN_GROUP.parseResponse(versionId, buffer), versionId); return new JoinGroupResponse(new JoinGroupResponseData(new ByteBufferAccessor(buffer), version));
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
} }
@Override @Override

View File

@ -24,8 +24,8 @@ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrParti
import org.apache.kafka.common.message.LeaderAndIsrResponseData; import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError; import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.FlattenedIterator; import org.apache.kafka.common.utils.FlattenedIterator;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -123,15 +123,6 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
} }
} }
public LeaderAndIsrRequest(Struct struct, short version) {
this(new LeaderAndIsrRequestData(struct, version), version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version());
}
@Override @Override
public LeaderAndIsrResponse getErrorResponse(int throttleTimeMs, Throwable e) { public LeaderAndIsrResponse getErrorResponse(int throttleTimeMs, Throwable e) {
LeaderAndIsrResponseData responseData = new LeaderAndIsrResponseData(); LeaderAndIsrResponseData responseData = new LeaderAndIsrResponseData();
@ -175,11 +166,11 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
return Collections.unmodifiableList(data.liveLeaders()); return Collections.unmodifiableList(data.liveLeaders());
} }
public LeaderAndIsrRequestData data() { protected LeaderAndIsrRequestData data() {
return data; return data;
} }
public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) { public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) {
return new LeaderAndIsrRequest(ApiKeys.LEADER_AND_ISR.parseRequest(version, buffer), version); return new LeaderAndIsrRequest(new LeaderAndIsrRequestData(new ByteBufferAccessor(buffer), version), version);
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.LeaderAndIsrResponseData; import org.apache.kafka.common.message.LeaderAndIsrResponseData;
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError; import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -38,13 +38,10 @@ public class LeaderAndIsrResponse extends AbstractResponse {
private final LeaderAndIsrResponseData data; private final LeaderAndIsrResponseData data;
public LeaderAndIsrResponse(LeaderAndIsrResponseData data) { public LeaderAndIsrResponse(LeaderAndIsrResponseData data) {
super(ApiKeys.LEADER_AND_ISR);
this.data = data; this.data = data;
} }
public LeaderAndIsrResponse(Struct struct, short version) {
this.data = new LeaderAndIsrResponseData(struct, version);
}
public List<LeaderAndIsrPartitionError> partitions() { public List<LeaderAndIsrPartitionError> partitions() {
return data.partitionErrors(); return data.partitionErrors();
} }
@ -65,13 +62,18 @@ public class LeaderAndIsrResponse extends AbstractResponse {
return errors; return errors;
} }
@Override
public int throttleTimeMs() {
return DEFAULT_THROTTLE_TIME;
}
public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) {
return new LeaderAndIsrResponse(ApiKeys.LEADER_AND_ISR.parseResponse(version, buffer), version); return new LeaderAndIsrResponse(new LeaderAndIsrResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override
protected Struct toStruct(short version) { protected LeaderAndIsrResponseData data() {
return data.toStruct(version); return data;
} }
@Override @Override

View File

@ -21,9 +21,9 @@ import org.apache.kafka.common.message.LeaveGroupRequestData;
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -81,18 +81,10 @@ public class LeaveGroupRequest extends AbstractRequest {
} }
} }
private final LeaveGroupRequestData data; private final LeaveGroupRequestData data;
private final short version;
private LeaveGroupRequest(LeaveGroupRequestData data, short version) { private LeaveGroupRequest(LeaveGroupRequestData data, short version) {
super(ApiKeys.LEAVE_GROUP, version); super(ApiKeys.LEAVE_GROUP, version);
this.data = data; this.data = data;
this.version = version;
}
public LeaveGroupRequest(Struct struct, short version) {
super(ApiKeys.LEAVE_GROUP, version);
this.data = new LeaveGroupRequestData(struct, version);
this.version = version;
} }
public LeaveGroupRequestData data() { public LeaveGroupRequestData data() {
@ -101,7 +93,7 @@ public class LeaveGroupRequest extends AbstractRequest {
public List<MemberIdentity> members() { public List<MemberIdentity> members() {
// Before version 3, leave group request is still in single mode // Before version 3, leave group request is still in single mode
return version <= 2 ? Collections.singletonList( return version() <= 2 ? Collections.singletonList(
new MemberIdentity() new MemberIdentity()
.setMemberId(data.memberId())) : data.members(); .setMemberId(data.memberId())) : data.members();
} }
@ -118,11 +110,6 @@ public class LeaveGroupRequest extends AbstractRequest {
} }
public static LeaveGroupRequest parse(ByteBuffer buffer, short version) { public static LeaveGroupRequest parse(ByteBuffer buffer, short version) {
return new LeaveGroupRequest(ApiKeys.LEAVE_GROUP.parseRequest(version, buffer), version); return new LeaveGroupRequest(new LeaveGroupRequestData(new ByteBufferAccessor(buffer), version), version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version);
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -51,6 +51,7 @@ public class LeaveGroupResponse extends AbstractResponse {
public final LeaveGroupResponseData data; public final LeaveGroupResponseData data;
public LeaveGroupResponse(LeaveGroupResponseData data) { public LeaveGroupResponse(LeaveGroupResponseData data) {
super(ApiKeys.LEAVE_GROUP);
this.data = data; this.data = data;
} }
@ -58,6 +59,7 @@ public class LeaveGroupResponse extends AbstractResponse {
Errors topLevelError, Errors topLevelError,
final int throttleTimeMs, final int throttleTimeMs,
final short version) { final short version) {
super(ApiKeys.LEAVE_GROUP);
if (version <= 2) { if (version <= 2) {
// Populate member level error. // Populate member level error.
final short errorCode = getError(topLevelError, memberResponses).code(); final short errorCode = getError(topLevelError, memberResponses).code();
@ -75,15 +77,6 @@ public class LeaveGroupResponse extends AbstractResponse {
} }
} }
public LeaveGroupResponse(Struct struct) {
short latestVersion = (short) (LeaveGroupResponseData.SCHEMAS.length - 1);
this.data = new LeaveGroupResponseData(struct, latestVersion);
}
public LeaveGroupResponse(Struct struct, short version) {
this.data = new LeaveGroupResponseData(struct, version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
@ -129,17 +122,12 @@ public class LeaveGroupResponse extends AbstractResponse {
} }
@Override @Override
public String toString() { protected LeaveGroupResponseData data() {
return data.toString(); return data;
} }
@Override public static LeaveGroupResponse parse(ByteBuffer buffer, short version) {
public Struct toStruct(short version) { return new LeaveGroupResponse(new LeaveGroupResponseData(new ByteBufferAccessor(buffer), version));
return data.toStruct(version);
}
public static LeaveGroupResponse parse(ByteBuffer buffer, short versionId) {
return new LeaveGroupResponse(ApiKeys.LEAVE_GROUP.parseResponse(versionId, buffer), versionId);
} }
@Override @Override
@ -157,4 +145,9 @@ public class LeaveGroupResponse extends AbstractResponse {
public int hashCode() { public int hashCode() {
return Objects.hashCode(data); return Objects.hashCode(data);
} }
@Override
public String toString() {
return data.toString();
}
} }

View File

@ -20,8 +20,8 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.ListGroupsRequestData; import org.apache.kafka.common.message.ListGroupsRequestData;
import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListGroupsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -66,15 +66,6 @@ public class ListGroupsRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public ListGroupsRequest(Struct struct, short version) {
super(ApiKeys.LIST_GROUPS, version);
this.data = new ListGroupsRequestData(struct, version);
}
public ListGroupsRequestData data() {
return data;
}
@Override @Override
public ListGroupsResponse getErrorResponse(int throttleTimeMs, Throwable e) { public ListGroupsResponse getErrorResponse(int throttleTimeMs, Throwable e) {
ListGroupsResponseData listGroupsResponseData = new ListGroupsResponseData(). ListGroupsResponseData listGroupsResponseData = new ListGroupsResponseData().
@ -87,11 +78,11 @@ public class ListGroupsRequest extends AbstractRequest {
} }
public static ListGroupsRequest parse(ByteBuffer buffer, short version) { public static ListGroupsRequest parse(ByteBuffer buffer, short version) {
return new ListGroupsRequest(ApiKeys.LIST_GROUPS.parseRequest(version, buffer), version); return new ListGroupsRequest(new ListGroupsRequestData(new ByteBufferAccessor(buffer), version), version);
} }
@Override @Override
protected Struct toStruct() { public ListGroupsRequestData data() {
return data.toStruct(version()); return data;
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListGroupsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -29,13 +29,10 @@ public class ListGroupsResponse extends AbstractResponse {
private final ListGroupsResponseData data; private final ListGroupsResponseData data;
public ListGroupsResponse(ListGroupsResponseData data) { public ListGroupsResponse(ListGroupsResponseData data) {
super(ApiKeys.LIST_GROUPS);
this.data = data; this.data = data;
} }
public ListGroupsResponse(Struct struct, short version) {
this.data = new ListGroupsResponseData(struct, version);
}
public ListGroupsResponseData data() { public ListGroupsResponseData data() {
return data; return data;
} }
@ -50,13 +47,8 @@ public class ListGroupsResponse extends AbstractResponse {
return errorCounts(Errors.forCode(data.errorCode())); return errorCounts(Errors.forCode(data.errorCode()));
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
public static ListGroupsResponse parse(ByteBuffer buffer, short version) { public static ListGroupsResponse parse(ByteBuffer buffer, short version) {
return new ListGroupsResponse(ApiKeys.LIST_GROUPS.responseSchema(version).read(buffer), version); return new ListGroupsResponse(new ListGroupsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -34,8 +34,8 @@ import org.apache.kafka.common.message.ListOffsetResponseData;
import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse;
import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
public class ListOffsetRequest extends AbstractRequest { public class ListOffsetRequest extends AbstractRequest {
public static final long EARLIEST_TIMESTAMP = -2L; public static final long EARLIEST_TIMESTAMP = -2L;
@ -80,7 +80,7 @@ public class ListOffsetRequest extends AbstractRequest {
@Override @Override
public ListOffsetRequest build(short version) { public ListOffsetRequest build(short version) {
return new ListOffsetRequest(version, data); return new ListOffsetRequest(data, version);
} }
@Override @Override
@ -92,15 +92,9 @@ public class ListOffsetRequest extends AbstractRequest {
/** /**
* Private constructor with a specified version. * Private constructor with a specified version.
*/ */
private ListOffsetRequest(short version, ListOffsetRequestData data) { private ListOffsetRequest(ListOffsetRequestData data, short version) {
super(ApiKeys.LIST_OFFSETS, version); super(ApiKeys.LIST_OFFSETS, version);
this.data = data; this.data = data;
this.duplicatePartitions = Collections.emptySet();
}
public ListOffsetRequest(Struct struct, short version) {
super(ApiKeys.LIST_OFFSETS, version);
data = new ListOffsetRequestData(struct, version);
duplicatePartitions = new HashSet<>(); duplicatePartitions = new HashSet<>();
Set<TopicPartition> partitions = new HashSet<>(); Set<TopicPartition> partitions = new HashSet<>();
for (ListOffsetTopic topic : data.topics()) { for (ListOffsetTopic topic : data.topics()) {
@ -143,6 +137,7 @@ public class ListOffsetRequest extends AbstractRequest {
return new ListOffsetResponse(responseData); return new ListOffsetResponse(responseData);
} }
@Override
public ListOffsetRequestData data() { public ListOffsetRequestData data() {
return data; return data;
} }
@ -164,12 +159,7 @@ public class ListOffsetRequest extends AbstractRequest {
} }
public static ListOffsetRequest parse(ByteBuffer buffer, short version) { public static ListOffsetRequest parse(ByteBuffer buffer, short version) {
return new ListOffsetRequest(ApiKeys.LIST_OFFSETS.parseRequest(version, buffer), version); return new ListOffsetRequest(new ListOffsetRequestData(new ByteBufferAccessor(buffer), version), version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version());
} }
public static List<ListOffsetTopic> toListOffsetTopics(Map<TopicPartition, ListOffsetPartition> timestampsToSearch) { public static List<ListOffsetTopic> toListOffsetTopics(Map<TopicPartition, ListOffsetPartition> timestampsToSearch) {

View File

@ -27,8 +27,8 @@ import org.apache.kafka.common.message.ListOffsetResponseData;
import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse;
import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
/** /**
@ -55,18 +55,16 @@ public class ListOffsetResponse extends AbstractResponse {
private final ListOffsetResponseData data; private final ListOffsetResponseData data;
public ListOffsetResponse(ListOffsetResponseData data) { public ListOffsetResponse(ListOffsetResponseData data) {
super(ApiKeys.LIST_OFFSETS);
this.data = data; this.data = data;
} }
public ListOffsetResponse(Struct struct, short version) {
data = new ListOffsetResponseData(struct, version);
}
@Override @Override
public int throttleTimeMs() { public int throttleTimeMs() {
return data.throttleTimeMs(); return data.throttleTimeMs();
} }
@Override
public ListOffsetResponseData data() { public ListOffsetResponseData data() {
return data; return data;
} }
@ -87,12 +85,7 @@ public class ListOffsetResponse extends AbstractResponse {
} }
public static ListOffsetResponse parse(ByteBuffer buffer, short version) { public static ListOffsetResponse parse(ByteBuffer buffer, short version) {
return new ListOffsetResponse(ApiKeys.LIST_OFFSETS.parseResponse(version, buffer), version); return new ListOffsetResponse(new ListOffsetResponseData(new ByteBufferAccessor(buffer), version));
}
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
} }
@Override @Override

View File

@ -21,7 +21,7 @@ import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData;
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingPartitionReassignment; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingPartitionReassignment;
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -52,38 +52,21 @@ public class ListPartitionReassignmentsRequest extends AbstractRequest {
} }
private ListPartitionReassignmentsRequestData data; private ListPartitionReassignmentsRequestData data;
private final short version;
private ListPartitionReassignmentsRequest(ListPartitionReassignmentsRequestData data, short version) { private ListPartitionReassignmentsRequest(ListPartitionReassignmentsRequestData data, short version) {
super(ApiKeys.LIST_PARTITION_REASSIGNMENTS, version); super(ApiKeys.LIST_PARTITION_REASSIGNMENTS, version);
this.data = data; this.data = data;
this.version = version;
}
ListPartitionReassignmentsRequest(Struct struct, short version) {
super(ApiKeys.LIST_PARTITION_REASSIGNMENTS, version);
this.data = new ListPartitionReassignmentsRequestData(struct, version);
this.version = version;
} }
public static ListPartitionReassignmentsRequest parse(ByteBuffer buffer, short version) { public static ListPartitionReassignmentsRequest parse(ByteBuffer buffer, short version) {
return new ListPartitionReassignmentsRequest( return new ListPartitionReassignmentsRequest(new ListPartitionReassignmentsRequestData(
ApiKeys.LIST_PARTITION_REASSIGNMENTS.parseRequest(version, buffer), version new ByteBufferAccessor(buffer), version), version);
);
} }
public ListPartitionReassignmentsRequestData data() { public ListPartitionReassignmentsRequestData data() {
return data; return data;
} }
/**
* Visible for testing.
*/
@Override
public Struct toStruct() {
return data.toStruct(version);
}
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
ApiError apiError = ApiError.fromThrowable(e); ApiError apiError = ApiError.fromThrowable(e);

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
@ -28,20 +28,14 @@ public class ListPartitionReassignmentsResponse extends AbstractResponse {
private final ListPartitionReassignmentsResponseData data; private final ListPartitionReassignmentsResponseData data;
public ListPartitionReassignmentsResponse(Struct struct) {
this(struct, ApiKeys.LIST_PARTITION_REASSIGNMENTS.latestVersion());
}
public ListPartitionReassignmentsResponse(ListPartitionReassignmentsResponseData responseData) { public ListPartitionReassignmentsResponse(ListPartitionReassignmentsResponseData responseData) {
super(ApiKeys.LIST_PARTITION_REASSIGNMENTS);
this.data = responseData; this.data = responseData;
} }
ListPartitionReassignmentsResponse(Struct struct, short version) {
this.data = new ListPartitionReassignmentsResponseData(struct, version);
}
public static ListPartitionReassignmentsResponse parse(ByteBuffer buffer, short version) { public static ListPartitionReassignmentsResponse parse(ByteBuffer buffer, short version) {
return new ListPartitionReassignmentsResponse(ApiKeys.LIST_PARTITION_REASSIGNMENTS.responseSchema(version).read(buffer), version); return new ListPartitionReassignmentsResponse(new ListPartitionReassignmentsResponseData(
new ByteBufferAccessor(buffer), version));
} }
public ListPartitionReassignmentsResponseData data() { public ListPartitionReassignmentsResponseData data() {
@ -62,9 +56,4 @@ public class ListPartitionReassignmentsResponse extends AbstractResponse {
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
return errorCounts(Errors.forCode(data.errorCode())); return errorCounts(Errors.forCode(data.errorCode()));
} }
@Override
protected Struct toStruct(short version) {
return data.toStruct(version);
}
} }

View File

@ -21,8 +21,8 @@ import org.apache.kafka.common.message.MetadataRequestData;
import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic; import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic;
import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.message.MetadataResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collection; import java.util.Collection;
@ -102,18 +102,10 @@ public class MetadataRequest extends AbstractRequest {
} }
private final MetadataRequestData data; private final MetadataRequestData data;
private final short version;
public MetadataRequest(MetadataRequestData data, short version) { public MetadataRequest(MetadataRequestData data, short version) {
super(ApiKeys.METADATA, version); super(ApiKeys.METADATA, version);
this.data = data; this.data = data;
this.version = version;
}
public MetadataRequest(Struct struct, short version) {
super(ApiKeys.METADATA, version);
this.data = new MetadataRequestData(struct, version);
this.version = version;
} }
public MetadataRequestData data() { public MetadataRequestData data() {
@ -125,7 +117,7 @@ public class MetadataRequest extends AbstractRequest {
Errors error = Errors.forException(e); Errors error = Errors.forException(e);
MetadataResponseData responseData = new MetadataResponseData(); MetadataResponseData responseData = new MetadataResponseData();
if (topics() != null) { if (topics() != null) {
for (String topic :topics()) for (String topic : topics())
responseData.topics().add(new MetadataResponseData.MetadataResponseTopic() responseData.topics().add(new MetadataResponseData.MetadataResponseTopic()
.setName(topic) .setName(topic)
.setErrorCode(error.code()) .setErrorCode(error.code())
@ -134,12 +126,12 @@ public class MetadataRequest extends AbstractRequest {
} }
responseData.setThrottleTimeMs(throttleTimeMs); responseData.setThrottleTimeMs(throttleTimeMs);
return new MetadataResponse(responseData); return new MetadataResponse(responseData, true);
} }
public boolean isAllTopics() { public boolean isAllTopics() {
return (data.topics() == null) || return (data.topics() == null) ||
(data.topics().isEmpty() && version == 0); //In version 0, an empty topic list indicates (data.topics().isEmpty() && version() == 0); //In version 0, an empty topic list indicates
// "request metadata for all topics." // "request metadata for all topics."
} }
@ -158,7 +150,7 @@ public class MetadataRequest extends AbstractRequest {
} }
public static MetadataRequest parse(ByteBuffer buffer, short version) { public static MetadataRequest parse(ByteBuffer buffer, short version) {
return new MetadataRequest(ApiKeys.METADATA.parseRequest(version, buffer), version); return new MetadataRequest(new MetadataRequestData(new ByteBufferAccessor(buffer), version), version);
} }
public static List<MetadataRequestTopic> convertToMetadataRequestTopic(final Collection<String> topics) { public static List<MetadataRequestTopic> convertToMetadataRequestTopic(final Collection<String> topics) {
@ -166,9 +158,4 @@ public class MetadataRequest extends AbstractRequest {
.setName(topic)) .setName(topic))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
@Override
protected Struct toStruct() {
return data.toStruct(version);
}
} }

View File

@ -25,9 +25,8 @@ import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseBrok
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition;
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -64,28 +63,19 @@ public class MetadataResponse extends AbstractResponse {
private volatile Holder holder; private volatile Holder holder;
private final boolean hasReliableLeaderEpochs; private final boolean hasReliableLeaderEpochs;
public MetadataResponse(MetadataResponseData data) { public MetadataResponse(MetadataResponseData data, short version) {
this(data, true); this(data, hasReliableLeaderEpochs(version));
} }
public MetadataResponse(Struct struct, short version) { MetadataResponse(MetadataResponseData data, boolean hasReliableLeaderEpochs) {
// Prior to Kafka version 2.4 (which coincides with Metadata version 9), the broker super(ApiKeys.METADATA);
// does not propagate leader epoch information accurately while a reassignment is in
// progress. Relying on a stale epoch can lead to FENCED_LEADER_EPOCH errors which
// can prevent consumption throughout the course of a reassignment. It is safer in
// this case to revert to the behavior in previous protocol versions which checks
// leader status only.
this(new MetadataResponseData(struct, version), version >= 9);
}
private MetadataResponse(MetadataResponseData data, boolean hasReliableLeaderEpochs) {
this.data = data; this.data = data;
this.hasReliableLeaderEpochs = hasReliableLeaderEpochs; this.hasReliableLeaderEpochs = hasReliableLeaderEpochs;
} }
@Override @Override
protected Struct toStruct(short version) { protected MetadataResponseData data() {
return data.toStruct(version); return data;
} }
@Override @Override
@ -243,8 +233,19 @@ public class MetadataResponse extends AbstractResponse {
return hasReliableLeaderEpochs; return hasReliableLeaderEpochs;
} }
// Prior to Kafka version 2.4 (which coincides with Metadata version 9), the broker
// does not propagate leader epoch information accurately while a reassignment is in
// progress. Relying on a stale epoch can lead to FENCED_LEADER_EPOCH errors which
// can prevent consumption throughout the course of a reassignment. It is safer in
// this case to revert to the behavior in previous protocol versions which checks
// leader status only.
private static boolean hasReliableLeaderEpochs(short version) {
return version >= 9;
}
public static MetadataResponse parse(ByteBuffer buffer, short version) { public static MetadataResponse parse(ByteBuffer buffer, short version) {
return new MetadataResponse(ApiKeys.METADATA.responseSchema(version).read(buffer), version); return new MetadataResponse(new MetadataResponseData(new ByteBufferAccessor(buffer), version),
hasReliableLeaderEpochs(version));
} }
public static class TopicMetadata { public static class TopicMetadata {
@ -429,79 +430,24 @@ public class MetadataResponse extends AbstractResponse {
} }
public static MetadataResponse prepareResponse(int throttleTimeMs, Collection<Node> brokers, String clusterId, public static MetadataResponse prepareResponse(short version,
int controllerId, List<TopicMetadata> topicMetadataList, int throttleTimeMs,
int clusterAuthorizedOperations,
short responseVersion) {
MetadataResponseData responseData = new MetadataResponseData();
responseData.setThrottleTimeMs(throttleTimeMs);
brokers.forEach(broker ->
responseData.brokers().add(new MetadataResponseBroker()
.setNodeId(broker.id())
.setHost(broker.host())
.setPort(broker.port())
.setRack(broker.rack()))
);
responseData.setClusterId(clusterId);
responseData.setControllerId(controllerId);
responseData.setClusterAuthorizedOperations(clusterAuthorizedOperations);
topicMetadataList.forEach(topicMetadata -> {
MetadataResponseTopic metadataResponseTopic = new MetadataResponseTopic();
metadataResponseTopic
.setErrorCode(topicMetadata.error.code())
.setName(topicMetadata.topic)
.setIsInternal(topicMetadata.isInternal)
.setTopicAuthorizedOperations(topicMetadata.authorizedOperations);
for (PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata) {
metadataResponseTopic.partitions().add(new MetadataResponsePartition()
.setErrorCode(partitionMetadata.error.code())
.setPartitionIndex(partitionMetadata.partition())
.setLeaderId(partitionMetadata.leaderId.orElse(NO_LEADER_ID))
.setLeaderEpoch(partitionMetadata.leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
.setReplicaNodes(partitionMetadata.replicaIds)
.setIsrNodes(partitionMetadata.inSyncReplicaIds)
.setOfflineReplicas(partitionMetadata.offlineReplicaIds));
}
responseData.topics().add(metadataResponseTopic);
});
return new MetadataResponse(responseData.toStruct(responseVersion), responseVersion);
}
public static MetadataResponse prepareResponse(int throttleTimeMs,
Collection<Node> brokers, Collection<Node> brokers,
String clusterId, String clusterId,
int controllerId, int controllerId,
List<TopicMetadata> topicMetadataList, List<MetadataResponseTopic> topics,
short responseVersion) { int clusterAuthorizedOperations) {
return prepareResponse(throttleTimeMs, brokers, clusterId, controllerId, topicMetadataList, return prepareResponse(hasReliableLeaderEpochs(version), throttleTimeMs, brokers, clusterId, controllerId,
MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED, responseVersion); topics, clusterAuthorizedOperations);
} }
public static MetadataResponse prepareResponse(Collection<Node> brokers, // Visible for testing
String clusterId, public static MetadataResponse prepareResponse(boolean hasReliableEpoch,
int controllerId, int throttleTimeMs,
List<TopicMetadata> topicMetadata,
short responseVersion) {
return prepareResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, brokers, clusterId, controllerId,
topicMetadata, responseVersion);
}
public static MetadataResponse prepareResponse(Collection<Node> brokers,
String clusterId,
int controllerId,
List<TopicMetadata> topicMetadata) {
return prepareResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, brokers, clusterId, controllerId,
topicMetadata, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse prepareResponse(int throttleTimeMs,
List<MetadataResponseTopic> topicMetadataList,
Collection<Node> brokers, Collection<Node> brokers,
String clusterId, String clusterId,
int controllerId, int controllerId,
List<MetadataResponseTopic> topics,
int clusterAuthorizedOperations) { int clusterAuthorizedOperations) {
MetadataResponseData responseData = new MetadataResponseData(); MetadataResponseData responseData = new MetadataResponseData();
responseData.setThrottleTimeMs(throttleTimeMs); responseData.setThrottleTimeMs(throttleTimeMs);
@ -517,8 +463,8 @@ public class MetadataResponse extends AbstractResponse {
responseData.setControllerId(controllerId); responseData.setControllerId(controllerId);
responseData.setClusterAuthorizedOperations(clusterAuthorizedOperations); responseData.setClusterAuthorizedOperations(clusterAuthorizedOperations);
topicMetadataList.forEach(topicMetadata -> responseData.topics().add(topicMetadata)); topics.forEach(topicMetadata -> responseData.topics().add(topicMetadata));
return new MetadataResponse(responseData); return new MetadataResponse(responseData, hasReliableEpoch);
} }
@Override @Override

View File

@ -24,8 +24,8 @@ import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition;
import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -68,19 +68,9 @@ public class OffsetCommitRequest extends AbstractRequest {
} }
} }
private final short version;
public OffsetCommitRequest(OffsetCommitRequestData data, short version) { public OffsetCommitRequest(OffsetCommitRequestData data, short version) {
super(ApiKeys.OFFSET_COMMIT, version); super(ApiKeys.OFFSET_COMMIT, version);
this.data = data; this.data = data;
this.version = version;
}
public OffsetCommitRequest(Struct struct, short version) {
super(ApiKeys.OFFSET_COMMIT, version);
this.data = new OffsetCommitRequestData(struct, version);
this.version = version;
} }
public OffsetCommitRequestData data() { public OffsetCommitRequestData data() {
@ -128,11 +118,6 @@ public class OffsetCommitRequest extends AbstractRequest {
} }
public static OffsetCommitRequest parse(ByteBuffer buffer, short version) { public static OffsetCommitRequest parse(ByteBuffer buffer, short version) {
return new OffsetCommitRequest(ApiKeys.OFFSET_COMMIT.parseRequest(version, buffer), version); return new OffsetCommitRequest(new OffsetCommitRequestData(new ByteBufferAccessor(buffer), version), version);
}
@Override
protected Struct toStruct() {
return data.toStruct(version);
} }
} }

View File

@ -21,8 +21,8 @@ import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition;
import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -50,10 +50,12 @@ public class OffsetCommitResponse extends AbstractResponse {
private final OffsetCommitResponseData data; private final OffsetCommitResponseData data;
public OffsetCommitResponse(OffsetCommitResponseData data) { public OffsetCommitResponse(OffsetCommitResponseData data) {
super(ApiKeys.OFFSET_COMMIT);
this.data = data; this.data = data;
} }
public OffsetCommitResponse(int requestThrottleMs, Map<TopicPartition, Errors> responseData) { public OffsetCommitResponse(int requestThrottleMs, Map<TopicPartition, Errors> responseData) {
super(ApiKeys.OFFSET_COMMIT);
Map<String, OffsetCommitResponseTopic> Map<String, OffsetCommitResponseTopic>
responseTopicDataMap = new HashMap<>(); responseTopicDataMap = new HashMap<>();
@ -79,15 +81,6 @@ public class OffsetCommitResponse extends AbstractResponse {
this(DEFAULT_THROTTLE_TIME, responseData); this(DEFAULT_THROTTLE_TIME, responseData);
} }
public OffsetCommitResponse(Struct struct) {
short latestVersion = (short) (OffsetCommitResponseData.SCHEMAS.length - 1);
this.data = new OffsetCommitResponseData(struct, latestVersion);
}
public OffsetCommitResponse(Struct struct, short version) {
this.data = new OffsetCommitResponseData(struct, version);
}
public OffsetCommitResponseData data() { public OffsetCommitResponseData data() {
return data; return data;
} }
@ -100,12 +93,7 @@ public class OffsetCommitResponse extends AbstractResponse {
} }
public static OffsetCommitResponse parse(ByteBuffer buffer, short version) { public static OffsetCommitResponse parse(ByteBuffer buffer, short version) {
return new OffsetCommitResponse(ApiKeys.OFFSET_COMMIT.parseResponse(version, buffer), version); return new OffsetCommitResponse(new OffsetCommitResponseData(new ByteBufferAccessor(buffer), version));
}
@Override
public Struct toStruct(short version) {
return data.toStruct(version);
} }
@Override @Override

View File

@ -19,8 +19,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.OffsetDeleteRequestData; import org.apache.kafka.common.message.OffsetDeleteRequestData;
import org.apache.kafka.common.message.OffsetDeleteResponseData; import org.apache.kafka.common.message.OffsetDeleteResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -53,11 +53,6 @@ public class OffsetDeleteRequest extends AbstractRequest {
this.data = data; this.data = data;
} }
public OffsetDeleteRequest(Struct struct, short version) {
super(ApiKeys.OFFSET_DELETE, version);
this.data = new OffsetDeleteRequestData(struct, version);
}
public AbstractResponse getErrorResponse(int throttleTimeMs, Errors error) { public AbstractResponse getErrorResponse(int throttleTimeMs, Errors error) {
return new OffsetDeleteResponse( return new OffsetDeleteResponse(
new OffsetDeleteResponseData() new OffsetDeleteResponseData()
@ -72,12 +67,11 @@ public class OffsetDeleteRequest extends AbstractRequest {
} }
public static OffsetDeleteRequest parse(ByteBuffer buffer, short version) { public static OffsetDeleteRequest parse(ByteBuffer buffer, short version) {
return new OffsetDeleteRequest(ApiKeys.OFFSET_DELETE.parseRequest(version, buffer), return new OffsetDeleteRequest(new OffsetDeleteRequestData(new ByteBufferAccessor(buffer), version), version);
version);
} }
@Override @Override
protected Struct toStruct() { protected OffsetDeleteRequestData data() {
return data.toStruct(version()); return data;
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.OffsetDeleteResponseData; import org.apache.kafka.common.message.OffsetDeleteResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
@ -47,21 +47,13 @@ public class OffsetDeleteResponse extends AbstractResponse {
public final OffsetDeleteResponseData data; public final OffsetDeleteResponseData data;
public OffsetDeleteResponse(OffsetDeleteResponseData data) { public OffsetDeleteResponse(OffsetDeleteResponseData data) {
super(ApiKeys.OFFSET_DELETE);
this.data = data; this.data = data;
} }
public OffsetDeleteResponse(Struct struct) {
short latestVersion = (short) (OffsetDeleteResponseData.SCHEMAS.length - 1);
this.data = new OffsetDeleteResponseData(struct, latestVersion);
}
public OffsetDeleteResponse(Struct struct, short version) {
this.data = new OffsetDeleteResponseData(struct, version);
}
@Override @Override
protected Struct toStruct(short version) { protected OffsetDeleteResponseData data() {
return data.toStruct(version); return data;
} }
@Override @Override
@ -77,7 +69,7 @@ public class OffsetDeleteResponse extends AbstractResponse {
} }
public static OffsetDeleteResponse parse(ByteBuffer buffer, short version) { public static OffsetDeleteResponse parse(ByteBuffer buffer, short version) {
return new OffsetDeleteResponse(ApiKeys.OFFSET_DELETE.parseResponse(version, buffer)); return new OffsetDeleteResponse(new OffsetDeleteResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

Some files were not shown because too many files have changed in this diff Show More