diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index b62c844f63b5a..950b03efe14fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.FetchRequestData; 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.record.MemoryRecords; import org.apache.kafka.common.record.RecordBatch; @@ -371,7 +372,7 @@ public String rackId() { } 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(new ByteBufferAccessor(buffer), version), version); } @Override