You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by li...@apache.org on 2018/09/09 07:17:51 UTC

[kafka] branch trunk updated: KAFKA-7333; Protocol changes for KIP-320

This is an automated email from the ASF dual-hosted git repository.

lindong pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 05ba5aa  KAFKA-7333; Protocol changes for KIP-320
05ba5aa is described below

commit 05ba5aa00847b18b74369a821e972bbba9f155eb
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Sun Sep 9 00:14:57 2018 -0700

    KAFKA-7333; Protocol changes for KIP-320
    
    This patch contains the protocol updates needed for KIP-320 as well as some of the basic consumer APIs (e.g. `OffsetAndMetadata` and `ConsumerRecord`). The inter-broker format version has not been changed and the brokers will continue to use the current API versions.
    
    Author: Jason Gustafson <ja...@confluent.io>
    
    Reviewers: Dong Lin <li...@gmail.com>
    
    Closes #5564 from hachikuji/KAFKA-7333
---
 .../kafka/clients/admin/KafkaAdminClient.java      |  11 +-
 .../kafka/clients/consumer/ConsumerRecord.java     |  54 +++-
 .../kafka/clients/consumer/OffsetAndMetadata.java  |  51 ++-
 .../kafka/clients/consumer/OffsetAndTimestamp.java |  47 ++-
 .../kafka/clients/consumer/StickyAssignor.java     |   2 +-
 .../consumer/internals/ConsumerCoordinator.java    |   6 +-
 .../consumer/internals/ConsumerProtocol.java       |   2 +-
 .../kafka/clients/consumer/internals/Fetcher.java  |  60 ++--
 .../producer/internals/TransactionManager.java     |   3 +-
 .../org/apache/kafka/common/PartitionInfo.java     |   8 +-
 .../org/apache/kafka/common/TopicPartition.java    |   1 +
 .../apache/kafka/common/protocol/CommonFields.java |  16 +-
 .../apache/kafka/common/protocol/types/Field.java  |  37 +++
 .../apache/kafka/common/protocol/types/Struct.java |  54 ++++
 .../kafka/common/requests/AbstractRequest.java     |   5 +-
 .../common/requests/AddOffsetsToTxnRequest.java    |   4 +-
 .../common/requests/AddPartitionsToTxnRequest.java |   6 +-
 .../requests/AddPartitionsToTxnResponse.java       |   2 +-
 .../kafka/common/requests/AlterConfigsRequest.java |   4 +-
 .../requests/AlterReplicaLogDirsRequest.java       |   6 +-
 .../requests/AlterReplicaLogDirsResponse.java      |   2 +-
 .../kafka/common/requests/ApiVersionsRequest.java  |   2 +-
 .../common/requests/ControlledShutdownRequest.java |   4 +-
 .../kafka/common/requests/CreateAclsRequest.java   |   4 +-
 .../requests/CreateDelegationTokenRequest.java     |   4 +-
 .../common/requests/CreatePartitionsRequest.java   |   4 +-
 .../kafka/common/requests/CreateTopicsRequest.java |   4 +-
 .../kafka/common/requests/DeleteAclsRequest.java   |   4 +-
 .../kafka/common/requests/DeleteGroupsRequest.java |   4 +-
 .../common/requests/DeleteRecordsRequest.java      |   6 +-
 .../common/requests/DeleteRecordsResponse.java     |   2 +-
 .../kafka/common/requests/DeleteTopicsRequest.java |   4 +-
 .../kafka/common/requests/DescribeAclsRequest.java |   4 +-
 .../common/requests/DescribeConfigsRequest.java    |   4 +-
 .../requests/DescribeDelegationTokenRequest.java   |   4 +-
 .../common/requests/DescribeGroupsRequest.java     |   4 +-
 .../common/requests/DescribeLogDirsRequest.java    |   4 +-
 .../common/requests/DescribeLogDirsResponse.java   |   2 +-
 .../kafka/common/requests/EndTxnRequest.java       |   4 +-
 .../kafka/common/requests/EpochEndOffset.java      |   1 -
 .../requests/ExpireDelegationTokenRequest.java     |   4 +-
 .../apache/kafka/common/requests/FetchRequest.java | 352 +++++++++++----------
 .../kafka/common/requests/FetchResponse.java       | 127 ++++----
 .../common/requests/FindCoordinatorRequest.java    |   4 +-
 .../kafka/common/requests/HeartbeatRequest.java    |   4 +-
 .../common/requests/InitProducerIdRequest.java     |   4 +-
 .../kafka/common/requests/JoinGroupRequest.java    |   4 +-
 .../kafka/common/requests/LeaderAndIsrRequest.java |   4 +-
 .../kafka/common/requests/LeaveGroupRequest.java   |   4 +-
 .../kafka/common/requests/ListGroupsRequest.java   |   4 +-
 .../kafka/common/requests/ListOffsetRequest.java   | 282 ++++++++---------
 .../kafka/common/requests/ListOffsetResponse.java  | 156 +++++----
 .../kafka/common/requests/MetadataRequest.java     |  36 ++-
 .../kafka/common/requests/MetadataResponse.java    | 319 ++++++++++---------
 .../kafka/common/requests/OffsetCommitRequest.java | 205 ++++++------
 .../common/requests/OffsetCommitResponse.java      |  83 ++---
 .../kafka/common/requests/OffsetFetchRequest.java  |  75 +++--
 .../kafka/common/requests/OffsetFetchResponse.java | 129 ++++----
 .../requests/OffsetsForLeaderEpochRequest.java     | 102 ++++--
 .../requests/OffsetsForLeaderEpochResponse.java    |  61 ++--
 .../kafka/common/requests/ProduceRequest.java      |   6 +-
 .../kafka/common/requests/ProduceResponse.java     |   2 +-
 .../requests/RenewDelegationTokenRequest.java      |   4 +-
 .../apache/kafka/common/requests/RequestUtils.java |  16 +
 .../common/requests/SaslAuthenticateRequest.java   |   4 +-
 .../common/requests/SaslHandshakeRequest.java      |   4 +-
 .../kafka/common/requests/StopReplicaRequest.java  |   4 +-
 .../kafka/common/requests/SyncGroupRequest.java    |   4 +-
 .../common/requests/TxnOffsetCommitRequest.java    | 107 ++++---
 .../common/requests/TxnOffsetCommitResponse.java   |  69 ++--
 .../common/requests/UpdateMetadataRequest.java     |   4 +-
 .../common/requests/WriteTxnMarkersRequest.java    |   6 +-
 .../common/requests/WriteTxnMarkersResponse.java   |   2 +-
 .../apache/kafka/common/utils/CollectionUtils.java |  29 +-
 .../kafka/clients/FetchSessionHandlerTest.java     |  73 +++--
 .../kafka/clients/admin/KafkaAdminClientTest.java  |  40 ++-
 .../kafka/clients/consumer/ConsumerRecordTest.java |   3 +
 .../kafka/clients/consumer/KafkaConsumerTest.java  |  22 +-
 ...etadataTest.java => OffsetAndMetadataTest.java} |  46 +--
 .../kafka/clients/consumer/StickyAssignorTest.java |   4 +-
 .../internals/ConsumerCoordinatorTest.java         |  44 +--
 .../clients/consumer/internals/FetcherTest.java    |  99 +++++-
 .../producer/internals/TransactionManagerTest.java |  22 +-
 ...cPartitionTest.java => TopicPartitionTest.java} |   3 +-
 .../kafka/common/requests/RequestResponseTest.java | 118 ++++---
 ...izedfile => offsetAndMetadataBeforeLeaderEpoch} | Bin
 .../offsetAndMetadataWithLeaderEpoch               | Bin 0 -> 257 bytes
 core/src/main/scala/kafka/api/ApiVersion.scala     |  11 +-
 .../coordinator/group/GroupMetadataManager.scala   |  18 +-
 .../src/main/scala/kafka/server/FetchSession.scala |   3 +-
 core/src/main/scala/kafka/server/KafkaApis.scala   |  45 +--
 .../main/scala/kafka/server/MetadataCache.scala    |  14 +-
 .../kafka/server/ReplicaAlterLogDirsThread.scala   |   4 +-
 .../scala/kafka/server/ReplicaFetcherThread.scala  |  38 ++-
 .../main/scala/kafka/server/ReplicaManager.scala   |   6 +-
 .../kafka/tools/ReplicaVerificationTool.scala      |   5 +-
 .../kafka/api/AuthorizerIntegrationTest.scala      |  13 +-
 .../test/scala/unit/kafka/api/ApiVersionTest.scala |   3 +-
 .../kafka/server/AbstractFetcherThreadTest.scala   |   4 +-
 .../FetchRequestDownConversionConfigTest.scala     |   5 +-
 .../scala/unit/kafka/server/FetchRequestTest.scala |   5 +-
 .../scala/unit/kafka/server/FetchSessionTest.scala |  29 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala    |  18 +-
 .../unit/kafka/server/ListOffsetsRequestTest.scala |  10 +-
 .../scala/unit/kafka/server/LogOffsetTest.scala    |  16 +-
 .../unit/kafka/server/MetadataCacheTest.scala      |   3 +-
 .../server/OffsetsForLeaderEpochRequestTest.scala  |   4 +-
 .../kafka/server/ReplicaFetcherThreadTest.scala    |   2 +-
 .../kafka/server/ReplicaManagerQuotasTest.scala    |  12 +-
 .../unit/kafka/server/ReplicaManagerTest.scala     |  57 ++--
 .../scala/unit/kafka/server/RequestQuotaTest.scala |  15 +-
 .../scala/unit/kafka/server/SimpleFetchTest.scala  |   5 +-
 .../server/epoch/LeaderEpochIntegrationTest.scala  |  11 +-
 .../server/epoch/OffsetsForLeaderEpochTest.scala   |  10 +-
 114 files changed, 2062 insertions(+), 1448 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index 904cd06..5759d63 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -134,6 +134,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -2643,12 +2644,14 @@ public class KafkaAdminClient extends AdminClient {
                             for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
                                     response.responseData().entrySet()) {
                                 final TopicPartition topicPartition = entry.getKey();
-                                final Errors error = entry.getValue().error;
+                                OffsetFetchResponse.PartitionData partitionData = entry.getValue();
+                                final Errors error = partitionData.error;
 
                                 if (error == Errors.NONE) {
-                                    final Long offset = entry.getValue().offset;
-                                    final String metadata = entry.getValue().metadata;
-                                    groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, metadata));
+                                    final Long offset = partitionData.offset;
+                                    final String metadata = partitionData.metadata;
+                                    final Optional<Integer> leaderEpoch = partitionData.leaderEpoch;
+                                    groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata));
                                 } else {
                                     log.warn("Skipping return offset for {} due to error {}.", topicPartition, error);
                                 }
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
index 7f85246..0413d5b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
@@ -22,6 +22,8 @@ import org.apache.kafka.common.record.DefaultRecord;
 import org.apache.kafka.common.record.RecordBatch;
 import org.apache.kafka.common.record.TimestampType;
 
+import java.util.Optional;
+
 /**
  * A key/value pair to be received from Kafka. This also consists of a topic name and 
  * a partition number from which the record is being received, an offset that points 
@@ -42,6 +44,7 @@ public class ConsumerRecord<K, V> {
     private final Headers headers;
     private final K key;
     private final V value;
+    private final Optional<Integer> leaderEpoch;
 
     private volatile Long checksum;
 
@@ -120,8 +123,41 @@ public class ConsumerRecord<K, V> {
                           K key,
                           V value,
                           Headers headers) {
+        this(topic, partition, offset, timestamp, timestampType, checksum, serializedKeySize, serializedValueSize,
+                key, value, headers, Optional.empty());
+    }
+
+    /**
+     * Creates a record to be received from a specified topic and partition
+     *
+     * @param topic The topic this record is received from
+     * @param partition The partition of the topic this record is received from
+     * @param offset The offset of this record in the corresponding Kafka partition
+     * @param timestamp The timestamp of the record.
+     * @param timestampType The timestamp type
+     * @param checksum The checksum (CRC32) of the full record
+     * @param serializedKeySize The length of the serialized key
+     * @param serializedValueSize The length of the serialized value
+     * @param key The key of the record, if one exists (null is allowed)
+     * @param value The record contents
+     * @param headers The headers of the record
+     * @param leaderEpoch Optional leader epoch of the record (may be empty for legacy record formats)
+     */
+    public ConsumerRecord(String topic,
+                          int partition,
+                          long offset,
+                          long timestamp,
+                          TimestampType timestampType,
+                          Long checksum,
+                          int serializedKeySize,
+                          int serializedValueSize,
+                          K key,
+                          V value,
+                          Headers headers,
+                          Optional<Integer> leaderEpoch) {
         if (topic == null)
             throw new IllegalArgumentException("Topic cannot be null");
+
         this.topic = topic;
         this.partition = partition;
         this.offset = offset;
@@ -133,6 +169,7 @@ public class ConsumerRecord<K, V> {
         this.key = key;
         this.value = value;
         this.headers = headers;
+        this.leaderEpoch = leaderEpoch;
     }
 
     /**
@@ -225,13 +262,26 @@ public class ConsumerRecord<K, V> {
         return this.serializedValueSize;
     }
 
+    /**
+     * Get the leader epoch for the record if available
+     *
+     * @return the leader epoch or empty for legacy record formats
+     */
+    public Optional<Integer> leaderEpoch() {
+        return leaderEpoch;
+    }
+
     @Override
     public String toString() {
-        return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset()
+        return "ConsumerRecord(topic = " + topic
+               + ", partition = " + partition
+               + ", leaderEpoch = " + leaderEpoch.orElse(null)
+               + ", offset = " + offset
                + ", " + timestampType + " = " + timestamp
                + ", serialized key size = "  + serializedKeySize
                + ", serialized value size = " + serializedValueSize
                + ", headers = " + headers
-               + ", key = " + key + ", value = " + value + ")";
+               + ", key = " + key
+               + ", value = " + value + ")";
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java
index 262d8f8..aa91e50 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java
@@ -19,6 +19,8 @@ package org.apache.kafka.clients.consumer;
 import org.apache.kafka.common.requests.OffsetFetchResponse;
 
 import java.io.Serializable;
+import java.util.Objects;
+import java.util.Optional;
 
 /**
  * The Kafka offset commit API allows users to provide additional metadata (in the form of a string)
@@ -26,16 +28,30 @@ import java.io.Serializable;
  * node made the commit, what time the commit was made, etc.
  */
 public class OffsetAndMetadata implements Serializable {
+    private static final long serialVersionUID = 2019555404968089681L;
+
     private final long offset;
     private final String metadata;
 
+    // We use null to represent the absence of a leader epoch to simplify serialization.
+    // I.e., older serializations of this class which do not have this field will automatically
+    // initialize its value to null.
+    private final Integer leaderEpoch;
+
     /**
      * Construct a new OffsetAndMetadata object for committing through {@link KafkaConsumer}.
+     *
      * @param offset The offset to be committed
+     * @param leaderEpoch Optional leader epoch of the last consumed record
      * @param metadata Non-null metadata
      */
-    public OffsetAndMetadata(long offset, String metadata) {
+    public OffsetAndMetadata(long offset, Optional<Integer> leaderEpoch, String metadata) {
+        if (offset < 0)
+            throw new IllegalArgumentException("Invalid negative offset");
+
         this.offset = offset;
+        this.leaderEpoch = leaderEpoch.orElse(null);
+
         // The server converts null metadata to an empty string. So we store it as an empty string as well on the client
         // to be consistent.
         if (metadata == null)
@@ -45,6 +61,15 @@ public class OffsetAndMetadata implements Serializable {
     }
 
     /**
+     * Construct a new OffsetAndMetadata object for committing through {@link KafkaConsumer}.
+     * @param offset The offset to be committed
+     * @param metadata Non-null metadata
+     */
+    public OffsetAndMetadata(long offset, String metadata) {
+        this(offset, Optional.empty(), metadata);
+    }
+
+    /**
      * Construct a new OffsetAndMetadata object for committing through {@link KafkaConsumer}. The metadata
      * associated with the commit will be empty.
      * @param offset The offset to be committed
@@ -61,29 +86,39 @@ public class OffsetAndMetadata implements Serializable {
         return metadata;
     }
 
+    /**
+     * Get the leader epoch of the previously consumed record (if one is known). Log truncation is detected
+     * if there exists a leader epoch which is larger than this epoch and begins at an offset earlier than
+     * the committed offset.
+     *
+     * @return the leader epoch or empty if not known
+     */
+    public Optional<Integer> leaderEpoch() {
+        return Optional.ofNullable(leaderEpoch);
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
-
         OffsetAndMetadata that = (OffsetAndMetadata) o;
-
-        if (offset != that.offset) return false;
-        return metadata.equals(that.metadata);
+        return offset == that.offset &&
+                Objects.equals(metadata, that.metadata) &&
+                Objects.equals(leaderEpoch, that.leaderEpoch);
     }
 
     @Override
     public int hashCode() {
-        int result = (int) (offset ^ (offset >>> 32));
-        result = 31 * result + metadata.hashCode();
-        return result;
+        return Objects.hash(offset, metadata, leaderEpoch);
     }
 
     @Override
     public String toString() {
         return "OffsetAndMetadata{" +
                 "offset=" + offset +
+                ", leaderEpoch=" + leaderEpoch +
                 ", metadata='" + metadata + '\'' +
                 '}';
     }
+
 }
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java
index 3af057f..40d9930 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java
@@ -16,7 +16,8 @@
  */
 package org.apache.kafka.clients.consumer;
 
-import org.apache.kafka.common.utils.Utils;
+import java.util.Objects;
+import java.util.Optional;
 
 /**
  * A container class for offset and timestamp.
@@ -24,12 +25,22 @@ import org.apache.kafka.common.utils.Utils;
 public final class OffsetAndTimestamp {
     private final long timestamp;
     private final long offset;
+    private final Optional<Integer> leaderEpoch;
 
     public OffsetAndTimestamp(long offset, long timestamp) {
+        this(offset, timestamp, Optional.empty());
+    }
+
+    public OffsetAndTimestamp(long offset, long timestamp, Optional<Integer> leaderEpoch) {
+        if (offset < 0)
+            throw new IllegalArgumentException("Invalid negative offset");
+
+        if (timestamp < 0)
+            throw new IllegalArgumentException("Invalid negative timestamp");
+
         this.offset = offset;
-        assert this.offset >= 0;
         this.timestamp = timestamp;
-        assert this.timestamp >= 0;
+        this.leaderEpoch = leaderEpoch;
     }
 
     public long timestamp() {
@@ -40,21 +51,35 @@ public final class OffsetAndTimestamp {
         return offset;
     }
 
+    /**
+     * Get the leader epoch corresponding to the offset that was found (if one exists).
+     * This can be provided to seek() to ensure that the log hasn't been truncated prior to fetching.
+     *
+     * @return The leader epoch or empty if it is not known
+     */
+    public Optional<Integer> leaderEpoch() {
+        return leaderEpoch;
+    }
+
     @Override
     public String toString() {
-        return "(timestamp=" + timestamp + ", offset=" + offset + ")";
+        return "(timestamp=" + timestamp +
+                ", leaderEpoch=" + leaderEpoch.orElse(null) +
+                ", offset=" + offset + ")";
     }
 
     @Override
-    public int hashCode() {
-        return 31 * Utils.longHashcode(timestamp) + Utils.longHashcode(offset);
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        OffsetAndTimestamp that = (OffsetAndTimestamp) o;
+        return timestamp == that.timestamp &&
+                offset == that.offset &&
+                Objects.equals(leaderEpoch, that.leaderEpoch);
     }
 
     @Override
-    public boolean equals(Object o) {
-        if (o == null || !(o instanceof OffsetAndTimestamp))
-            return false;
-        OffsetAndTimestamp other = (OffsetAndTimestamp) o;
-        return this.timestamp == other.timestamp() && this.offset == other.offset();
+    public int hashCode() {
+        return Objects.hash(timestamp, offset, leaderEpoch);
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
index 12da7e5..0d74eed 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
@@ -671,7 +671,7 @@ public class StickyAssignor extends AbstractPartitionAssignor {
     static ByteBuffer serializeTopicPartitionAssignment(List<TopicPartition> partitions) {
         Struct struct = new Struct(STICKY_ASSIGNOR_USER_DATA);
         List<Struct> topicAssignments = new ArrayList<>();
-        for (Map.Entry<String, List<Integer>> topicEntry : CollectionUtils.groupDataByTopic(partitions).entrySet()) {
+        for (Map.Entry<String, List<Integer>> topicEntry : CollectionUtils.groupPartitionsByTopic(partitions).entrySet()) {
             Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT);
             topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey());
             topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray());
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index ce2db35..3b0a5fa 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -760,8 +760,8 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
             if (offsetAndMetadata.offset() < 0) {
                 return RequestFuture.failure(new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()));
             }
-            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(
-                    offsetAndMetadata.offset(), offsetAndMetadata.metadata()));
+            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(offsetAndMetadata.offset(),
+                    offsetAndMetadata.leaderEpoch(), offsetAndMetadata.metadata()));
         }
 
         final Generation generation;
@@ -913,7 +913,7 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
                     return;
                 } else if (data.offset >= 0) {
                     // record the position with the offset (-1 indicates no committed offset to fetch)
-                    offsets.put(tp, new OffsetAndMetadata(data.offset, data.metadata));
+                    offsets.put(tp, new OffsetAndMetadata(data.offset, data.leaderEpoch, data.metadata));
                 } else {
                     log.debug("Found no committed offset for partition {}", tp);
                 }
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java
index 920c295..8a4aef8 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java
@@ -123,7 +123,7 @@ public class ConsumerProtocol {
         Struct struct = new Struct(ASSIGNMENT_V0);
         struct.set(USER_DATA_KEY_NAME, assignment.userData());
         List<Struct> topicAssignments = new ArrayList<>();
-        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignment.partitions());
+        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupPartitionsByTopic(assignment.partitions());
         for (Map.Entry<String, List<Integer>> topicEntry : partitionsByTopic.entrySet()) {
             Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT_V0);
             topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey());
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index 36a3314..dc0daa2 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -82,6 +82,7 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -168,10 +169,12 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
     private static class OffsetData {
         final long offset;
         final Long timestamp; //  null if the broker does not support returning timestamps
+        final Optional<Integer> leaderEpoch; // empty if the leader epoch is not known
 
-        OffsetData(long offset, Long timestamp) {
+        OffsetData(long offset, Long timestamp, Optional<Integer> leaderEpoch) {
             this.offset = offset;
             this.timestamp = timestamp;
+            this.leaderEpoch = leaderEpoch;
         }
     }
 
@@ -383,7 +386,8 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
             // 'entry.getValue().timestamp' will not be null since we are guaranteed
             // to work with a v1 (or later) ListOffset request
             OffsetData offsetData = entry.getValue();
-            offsetsByTimes.put(entry.getKey(), new OffsetAndTimestamp(offsetData.offset, offsetData.timestamp));
+            offsetsByTimes.put(entry.getKey(), new OffsetAndTimestamp(offsetData.offset, offsetData.timestamp,
+                    offsetData.leaderEpoch));
         }
 
         return offsetsByTimes;
@@ -570,10 +574,11 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
         for (TopicPartition tp : partitionResetTimestamps.keySet())
             metadata.add(tp.topic());
 
-        Map<Node, Map<TopicPartition, Long>> timestampsToSearchByNode = groupListOffsetRequests(partitionResetTimestamps);
-        for (Map.Entry<Node, Map<TopicPartition, Long>> entry : timestampsToSearchByNode.entrySet()) {
+        Map<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> timestampsToSearchByNode =
+                groupListOffsetRequests(partitionResetTimestamps);
+        for (Map.Entry<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> entry : timestampsToSearchByNode.entrySet()) {
             Node node = entry.getKey();
-            final Map<TopicPartition, Long> resetTimestamps = entry.getValue();
+            final Map<TopicPartition, ListOffsetRequest.PartitionData> resetTimestamps = entry.getValue();
             subscriptions.setResetPending(resetTimestamps.keySet(), time.milliseconds() + requestTimeoutMs);
 
             RequestFuture<ListOffsetResult> future = sendListOffsetRequest(node, resetTimestamps, false);
@@ -588,8 +593,8 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                     for (Map.Entry<TopicPartition, OffsetData> fetchedOffset : result.fetchedOffsets.entrySet()) {
                         TopicPartition partition = fetchedOffset.getKey();
                         OffsetData offsetData = fetchedOffset.getValue();
-                        Long requestedResetTimestamp = resetTimestamps.get(partition);
-                        resetOffsetIfNeeded(partition, requestedResetTimestamp, offsetData);
+                        ListOffsetRequest.PartitionData requestedReset = resetTimestamps.get(partition);
+                        resetOffsetIfNeeded(partition, requestedReset.timestamp, offsetData);
                     }
                 }
 
@@ -619,7 +624,8 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
         for (TopicPartition tp : timestampsToSearch.keySet())
             metadata.add(tp.topic());
 
-        Map<Node, Map<TopicPartition, Long>> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch);
+        Map<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> timestampsToSearchByNode =
+                groupListOffsetRequests(timestampsToSearch);
         if (timestampsToSearchByNode.isEmpty())
             return RequestFuture.failure(new StaleMetadataException());
 
@@ -628,7 +634,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
         final Set<TopicPartition> partitionsToRetry = new HashSet<>();
         final AtomicInteger remainingResponses = new AtomicInteger(timestampsToSearchByNode.size());
 
-        for (Map.Entry<Node, Map<TopicPartition, Long>> entry : timestampsToSearchByNode.entrySet()) {
+        for (Map.Entry<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> entry : timestampsToSearchByNode.entrySet()) {
             RequestFuture<ListOffsetResult> future =
                     sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps);
             future.addListener(new RequestFutureListener<ListOffsetResult>() {
@@ -657,8 +663,9 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
         return listOffsetRequestsFuture;
     }
 
-    private Map<Node, Map<TopicPartition, Long>> groupListOffsetRequests(Map<TopicPartition, Long> timestampsToSearch) {
-        final Map<Node, Map<TopicPartition, Long>> timestampsToSearchByNode = new HashMap<>();
+    private Map<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> groupListOffsetRequests(
+            Map<TopicPartition, Long> timestampsToSearch) {
+        final Map<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> timestampsToSearchByNode = new HashMap<>();
         for (Map.Entry<TopicPartition, Long> entry: timestampsToSearch.entrySet()) {
             TopicPartition tp  = entry.getKey();
             PartitionInfo info = metadata.fetch().partition(tp);
@@ -679,12 +686,11 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                         info.leader(), tp);
             } else {
                 Node node = info.leader();
-                Map<TopicPartition, Long> topicData = timestampsToSearchByNode.get(node);
-                if (topicData == null) {
-                    topicData = new HashMap<>();
-                    timestampsToSearchByNode.put(node, topicData);
-                }
-                topicData.put(entry.getKey(), entry.getValue());
+                Map<TopicPartition, ListOffsetRequest.PartitionData> topicData =
+                        timestampsToSearchByNode.computeIfAbsent(node, n -> new HashMap<>());
+                ListOffsetRequest.PartitionData partitionData = new ListOffsetRequest.PartitionData(
+                        entry.getValue(), Optional.empty());
+                topicData.put(entry.getKey(), partitionData);
             }
         }
         return timestampsToSearchByNode;
@@ -699,7 +705,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
      * @return A response which can be polled to obtain the corresponding timestamps and offsets.
      */
     private RequestFuture<ListOffsetResult> sendListOffsetRequest(final Node node,
-                                                                  final Map<TopicPartition, Long> timestampsToSearch,
+                                                                  final Map<TopicPartition, ListOffsetRequest.PartitionData> timestampsToSearch,
                                                                   boolean requireTimestamp) {
         ListOffsetRequest.Builder builder = ListOffsetRequest.Builder
                 .forConsumer(requireTimestamp, isolationLevel)
@@ -729,14 +735,14 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
      *               return a null timestamp (-1 is returned instead when necessary).
      */
     @SuppressWarnings("deprecation")
-    private void handleListOffsetResponse(Map<TopicPartition, Long> timestampsToSearch,
+    private void handleListOffsetResponse(Map<TopicPartition, ListOffsetRequest.PartitionData> timestampsToSearch,
                                           ListOffsetResponse listOffsetResponse,
                                           RequestFuture<ListOffsetResult> future) {
         Map<TopicPartition, OffsetData> fetchedOffsets = new HashMap<>();
         Set<TopicPartition> partitionsToRetry = new HashSet<>();
         Set<String> unauthorizedTopics = new HashSet<>();
 
-        for (Map.Entry<TopicPartition, Long> entry : timestampsToSearch.entrySet()) {
+        for (Map.Entry<TopicPartition, ListOffsetRequest.PartitionData> entry : timestampsToSearch.entrySet()) {
             TopicPartition topicPartition = entry.getKey();
             ListOffsetResponse.PartitionData partitionData = listOffsetResponse.responseData().get(topicPartition);
             Errors error = partitionData.error;
@@ -756,7 +762,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                     log.debug("Handling v0 ListOffsetResponse response for {}. Fetched offset {}",
                             topicPartition, offset);
                     if (offset != ListOffsetResponse.UNKNOWN_OFFSET) {
-                        OffsetData offsetData = new OffsetData(offset, null);
+                        OffsetData offsetData = new OffsetData(offset, null, Optional.empty());
                         fetchedOffsets.put(topicPartition, offsetData);
                     }
                 } else {
@@ -764,7 +770,8 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                     log.debug("Handling ListOffsetResponse response for {}. Fetched offset {}, timestamp {}",
                             topicPartition, partitionData.offset, partitionData.timestamp);
                     if (partitionData.offset != ListOffsetResponse.UNKNOWN_OFFSET) {
-                        OffsetData offsetData = new OffsetData(partitionData.offset, partitionData.timestamp);
+                        OffsetData offsetData = new OffsetData(partitionData.offset, partitionData.timestamp,
+                                partitionData.leaderEpoch);
                         fetchedOffsets.put(topicPartition, offsetData);
                     }
                 }
@@ -857,7 +864,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
 
                 long position = this.subscriptions.position(partition);
                 builder.add(partition, new FetchRequest.PartitionData(position, FetchRequest.INVALID_LOG_START_OFFSET,
-                    this.fetchSize));
+                    this.fetchSize, Optional.empty()));
 
                 log.debug("Added {} fetch request for partition {} at offset {} to node {}", isolationLevel,
                     partition, position, node);
@@ -979,6 +986,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
         try {
             long offset = record.offset();
             long timestamp = record.timestamp();
+            Optional<Integer> leaderEpoch = maybeLeaderEpoch(batch.partitionLeaderEpoch());
             TimestampType timestampType = batch.timestampType();
             Headers headers = new RecordHeaders(record.headers());
             ByteBuffer keyBytes = record.key();
@@ -991,13 +999,17 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                                         timestamp, timestampType, record.checksumOrNull(),
                                         keyByteArray == null ? ConsumerRecord.NULL_SIZE : keyByteArray.length,
                                         valueByteArray == null ? ConsumerRecord.NULL_SIZE : valueByteArray.length,
-                                        key, value, headers);
+                                        key, value, headers, leaderEpoch);
         } catch (RuntimeException e) {
             throw new SerializationException("Error deserializing key/value for partition " + partition +
                     " at offset " + record.offset() + ". If needed, please seek past the record to continue consumption.", e);
         }
     }
 
+    private Optional<Integer> maybeLeaderEpoch(int leaderEpoch) {
+        return leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH ? Optional.empty() : Optional.of(leaderEpoch);
+    }
+
     @Override
     public void onAssignment(Set<TopicPartition> assignment) {
         sensors.updatePartitionLagAndLeadSensors(assignment);
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index c0685c9..2cbd1e9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -839,7 +839,8 @@ public class TransactionManager {
                                                           String consumerGroupId) {
         for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
             OffsetAndMetadata offsetAndMetadata = entry.getValue();
-            CommittedOffset committedOffset = new CommittedOffset(offsetAndMetadata.offset(), offsetAndMetadata.metadata());
+            CommittedOffset committedOffset = new CommittedOffset(offsetAndMetadata.offset(),
+                    offsetAndMetadata.metadata(), offsetAndMetadata.leaderEpoch());
             pendingTxnOffsetCommits.put(entry.getKey(), committedOffset);
         }
         TxnOffsetCommitRequest.Builder builder = new TxnOffsetCommitRequest.Builder(transactionalId, consumerGroupId,
diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
index 38e4f67..44cd4f4 100644
--- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
+++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
@@ -20,7 +20,6 @@ package org.apache.kafka.common;
  * This is used to describe per-partition state in the MetadataResponse.
  */
 public class PartitionInfo {
-
     private final String topic;
     private final int partition;
     private final Node leader;
@@ -33,7 +32,12 @@ public class PartitionInfo {
         this(topic, partition, leader, replicas, inSyncReplicas, new Node[0]);
     }
 
-    public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas, Node[] offlineReplicas) {
+    public PartitionInfo(String topic,
+                         int partition,
+                         Node leader,
+                         Node[] replicas,
+                         Node[] inSyncReplicas,
+                         Node[] offlineReplicas) {
         this.topic = topic;
         this.partition = partition;
         this.leader = leader;
diff --git a/clients/src/main/java/org/apache/kafka/common/TopicPartition.java b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java
index dc79c2e..08b2a51 100644
--- a/clients/src/main/java/org/apache/kafka/common/TopicPartition.java
+++ b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
  * A topic name and partition number
  */
 public final class TopicPartition implements Serializable {
+    private static final long serialVersionUID = -613627415771699627L;
 
     private int hash = 0;
     private final int partition;
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
index 9eddf2b..708500c 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
@@ -27,7 +27,12 @@ public class CommonFields {
     public static final Field.Int32 PARTITION_ID = new Field.Int32("partition", "Topic partition id");
     public static final Field.Int16 ERROR_CODE = new Field.Int16("error_code", "Response error code");
     public static final Field.NullableStr ERROR_MESSAGE = new Field.NullableStr("error_message", "Response error message");
-    public static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The epoch");
+    public static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The leader epoch");
+    public static final Field.Int32 CURRENT_LEADER_EPOCH = new Field.Int32("current_leader_epoch",
+            "The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. " +
+                    "If the epoch provided by the client is larger than the current epoch known to the broker, then " +
+                    "the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then " +
+                    "the FENCED_LEADER_EPOCH error code will be returned.");
 
     // Group APIs
     public static final Field.Str GROUP_ID = new Field.Str("group_id", "The unique group identifier");
@@ -58,4 +63,13 @@ public class CommonFields {
     public static final Field.Str PRINCIPAL_TYPE = new Field.Str("principal_type", "principalType of the Kafka principal");
     public static final Field.Str PRINCIPAL_NAME = new Field.Str("name", "name of the Kafka principal");
 
+    public static final Field.Int64 COMMITTED_OFFSET = new Field.Int64("offset",
+            "Message offset to be committed");
+    public static final Field.NullableStr COMMITTED_METADATA = new Field.NullableStr("metadata",
+            "Any associated metadata the client wants to keep.");
+    public static final Field.Int32 COMMITTED_LEADER_EPOCH = new Field.Int32("leader_epoch",
+            "The leader epoch, if provided is derived from the last consumed record. " +
+                    "This is used by the consumer to check for log truncation and to ensure partition " +
+                    "metadata is up to date following a group rebalance.");
+
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
index 5c17001..72e051c 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
@@ -92,4 +92,41 @@ public class Field {
             super(name, Type.NULLABLE_STRING, docString, false, null);
         }
     }
+
+    public static class Bool extends Field {
+        public Bool(String name, String docString) {
+            super(name, Type.BOOLEAN, docString, false, null);
+        }
+    }
+
+    public static class Array extends Field {
+        public Array(String name, Type elementType, String docString) {
+            super(name, new ArrayOf(elementType), docString, false, null);
+        }
+    }
+
+    public static class ComplexArray {
+        public final String name;
+        public final String docString;
+
+        public ComplexArray(String name, String docString) {
+            this.name = name;
+            this.docString = docString;
+        }
+
+        public Field withFields(Field... fields) {
+            Schema elementType = new Schema(fields);
+            return new Field(name, new ArrayOf(elementType), docString, false, null);
+        }
+
+        public Field nullableWithFields(Field... fields) {
+            Schema elementType = new Schema(fields);
+            return new Field(name, ArrayOf.nullable(elementType), docString, false, null);
+        }
+
+        public Field withFields(String docStringOverride, Field... fields) {
+            Schema elementType = new Schema(fields);
+            return new Field(name, new ArrayOf(elementType), docStringOverride, false, null);
+        }
+    }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index 7183aed..94d5ae1 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -99,6 +99,14 @@ public class Struct {
         return getString(field.name);
     }
 
+    public Object[] get(Field.Array field) {
+        return getArray(field.name);
+    }
+
+    public Object[] get(Field.ComplexArray field) {
+        return getArray(field.name);
+    }
+
     public Long getOrElse(Field.Int64 field, long alternative) {
         if (hasField(field.name))
             return getLong(field.name);
@@ -135,6 +143,24 @@ public class Struct {
         return alternative;
     }
 
+    public boolean getOrElse(Field.Bool field, boolean alternative) {
+        if (hasField(field.name))
+            return getBoolean(field.name);
+        return alternative;
+    }
+
+    public Object[] getOrEmpty(Field.Array field) {
+        if (hasField(field.name))
+            return getArray(field.name);
+        return new Object[0];
+    }
+
+    public Object[] getOrEmpty(Field.ComplexArray field) {
+        if (hasField(field.name))
+            return getArray(field.name);
+        return new Object[0];
+    }
+
     /**
      * Get the record value for the field with the given name by doing a hash table lookup (slower!)
      *
@@ -162,6 +188,10 @@ public class Struct {
         return schema.get(def.name) != null;
     }
 
+    public boolean hasField(Field.ComplexArray def) {
+        return schema.get(def.name) != null;
+    }
+
     public Struct getStruct(BoundField field) {
         return (Struct) get(field);
     }
@@ -300,6 +330,22 @@ public class Struct {
         return set(def.name, value);
     }
 
+    public Struct set(Field.Array def, Object[] value) {
+        return set(def.name, value);
+    }
+
+    public Struct set(Field.ComplexArray def, Object[] value) {
+        return set(def.name, value);
+    }
+
+    public Struct setIfExists(Field.Array def, Object[] value) {
+        return set(def.name, value);
+    }
+
+    public Struct setIfExists(Field.ComplexArray def, Object[] value) {
+        return set(def.name, value);
+    }
+
     public Struct setIfExists(Field def, Object value) {
         return setIfExists(def.name, value);
     }
@@ -343,6 +389,14 @@ public class Struct {
         return instance(schema.get(field));
     }
 
+    public Struct instance(Field field) {
+        return instance(schema.get(field.name));
+    }
+
+    public Struct instance(Field.ComplexArray field) {
+        return instance(schema.get(field.name));
+    }
+
     /**
      * Empty all the values from this record
      */
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
index d2b93c4..d16e60f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.common.requests;
 
+import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.network.NetworkSend;
 import org.apache.kafka.common.network.Send;
 import org.apache.kafka.common.protocol.ApiKeys;
@@ -76,7 +77,9 @@ public abstract class AbstractRequest extends AbstractRequestResponse {
 
     private final short version;
 
-    public AbstractRequest(short version) {
+    public AbstractRequest(ApiKeys api, short version) {
+        if (!api.isVersionSupported(version))
+            throw new UnsupportedVersionException("The " + api + " protocol does not support version " + version);
         this.version = version;
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
index 6fb9441..2668ae1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
@@ -86,7 +86,7 @@ public class AddOffsetsToTxnRequest extends AbstractRequest {
     private final String consumerGroupId;
 
     private AddOffsetsToTxnRequest(short version, String transactionalId, long producerId, short producerEpoch, String consumerGroupId) {
-        super(version);
+        super(ApiKeys.ADD_OFFSETS_TO_TXN, version);
         this.transactionalId = transactionalId;
         this.producerId = producerId;
         this.producerEpoch = producerEpoch;
@@ -94,7 +94,7 @@ public class AddOffsetsToTxnRequest extends AbstractRequest {
     }
 
     public AddOffsetsToTxnRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.ADD_OFFSETS_TO_TXN, version);
         this.transactionalId = struct.get(TRANSACTIONAL_ID);
         this.producerId = struct.get(PRODUCER_ID);
         this.producerEpoch = struct.get(PRODUCER_EPOCH);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
index 4a87289..3be7c98 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
@@ -102,7 +102,7 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
 
     private AddPartitionsToTxnRequest(short version, String transactionalId, long producerId, short producerEpoch,
                                       List<TopicPartition> partitions) {
-        super(version);
+        super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.transactionalId = transactionalId;
         this.producerId = producerId;
         this.producerEpoch = producerEpoch;
@@ -110,7 +110,7 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     }
 
     public AddPartitionsToTxnRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.transactionalId = struct.get(TRANSACTIONAL_ID);
         this.producerId = struct.get(PRODUCER_ID);
         this.producerEpoch = struct.get(PRODUCER_EPOCH);
@@ -150,7 +150,7 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
         struct.set(PRODUCER_ID, producerId);
         struct.set(PRODUCER_EPOCH, producerEpoch);
 
-        Map<String, List<Integer>> mappedPartitions = CollectionUtils.groupDataByTopic(partitions);
+        Map<String, List<Integer>> mappedPartitions = CollectionUtils.groupPartitionsByTopic(partitions);
         Object[] partitionsArray = new Object[mappedPartitions.size()];
         int i = 0;
         for (Map.Entry<String, List<Integer>> topicAndPartitions : mappedPartitions.entrySet()) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
index 977bd59..ea8a073 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
@@ -110,7 +110,7 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
         Struct struct = new Struct(ApiKeys.ADD_PARTITIONS_TO_TXN.responseSchema(version));
         struct.set(THROTTLE_TIME_MS, throttleTimeMs);
 
-        Map<String, Map<Integer, Errors>> errorsByTopic = CollectionUtils.groupDataByTopic(errors);
+        Map<String, Map<Integer, Errors>> errorsByTopic = CollectionUtils.groupPartitionDataByTopic(errors);
         List<Struct> topics = new ArrayList<>(errorsByTopic.size());
         for (Map.Entry<String, Map<Integer, Errors>> entry : errorsByTopic.entrySet()) {
             Struct topicErrorCodes = struct.instance(ERRORS_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
index ff1d062..963ad06 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
@@ -123,13 +123,13 @@ public class AlterConfigsRequest extends AbstractRequest {
     private final boolean validateOnly;
 
     public AlterConfigsRequest(short version, Map<ConfigResource, Config> configs, boolean validateOnly) {
-        super(version);
+        super(ApiKeys.ALTER_CONFIGS, version);
         this.configs = Objects.requireNonNull(configs, "configs");
         this.validateOnly = validateOnly;
     }
 
     public AlterConfigsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.ALTER_CONFIGS, version);
         validateOnly = struct.getBoolean(VALIDATE_ONLY_KEY_NAME);
         Object[] resourcesArray = struct.getArray(RESOURCES_KEY_NAME);
         configs = new HashMap<>(resourcesArray.length);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java
index 0bc7cd0..03bc098 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java
@@ -91,7 +91,7 @@ public class AlterReplicaLogDirsRequest extends AbstractRequest {
     }
 
     public AlterReplicaLogDirsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.ALTER_REPLICA_LOG_DIRS, version);
         partitionDirs = new HashMap<>();
         for (Object logDirStructObj : struct.getArray(LOG_DIRS_KEY_NAME)) {
             Struct logDirStruct = (Struct) logDirStructObj;
@@ -108,7 +108,7 @@ public class AlterReplicaLogDirsRequest extends AbstractRequest {
     }
 
     public AlterReplicaLogDirsRequest(Map<TopicPartition, String> partitionDirs, short version) {
-        super(version);
+        super(ApiKeys.ALTER_REPLICA_LOG_DIRS, version);
         this.partitionDirs = partitionDirs;
     }
 
@@ -128,7 +128,7 @@ public class AlterReplicaLogDirsRequest extends AbstractRequest {
             logDirStruct.set(LOG_DIR_KEY_NAME, logDirEntry.getKey());
 
             List<Struct> topicStructArray = new ArrayList<>();
-            for (Map.Entry<String, List<Integer>> topicEntry: CollectionUtils.groupDataByTopic(logDirEntry.getValue()).entrySet()) {
+            for (Map.Entry<String, List<Integer>> topicEntry: CollectionUtils.groupPartitionsByTopic(logDirEntry.getValue()).entrySet()) {
                 Struct topicStruct = logDirStruct.instance(TOPICS_KEY_NAME);
                 topicStruct.set(TOPIC_NAME, topicEntry.getKey());
                 topicStruct.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray());
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java
index c8f6e4d..7a73275 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java
@@ -101,7 +101,7 @@ public class AlterReplicaLogDirsResponse extends AbstractResponse {
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.ALTER_REPLICA_LOG_DIRS.responseSchema(version));
         struct.set(THROTTLE_TIME_MS, throttleTimeMs);
-        Map<String, Map<Integer, Errors>> responsesByTopic = CollectionUtils.groupDataByTopic(responses);
+        Map<String, Map<Integer, Errors>> responsesByTopic = CollectionUtils.groupPartitionDataByTopic(responses);
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Errors>> responsesByTopicEntry : responsesByTopic.entrySet()) {
             Struct topicStruct = struct.instance(TOPICS_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
index 347e355..e154ac9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
@@ -67,7 +67,7 @@ public class ApiVersionsRequest extends AbstractRequest {
     }
 
     public ApiVersionsRequest(short version, Short unsupportedRequestVersion) {
-        super(version);
+        super(ApiKeys.API_VERSIONS, version);
 
         // Unlike other request types, the broker handles ApiVersion requests with higher versions than
         // supported. It does so by treating the request as if it were v0 and returns a response using
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
index e6e8734..d6db1e1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
@@ -64,12 +64,12 @@ public class ControlledShutdownRequest extends AbstractRequest {
     private final int brokerId;
 
     private ControlledShutdownRequest(int brokerId, short version) {
-        super(version);
+        super(ApiKeys.CONTROLLED_SHUTDOWN, version);
         this.brokerId = brokerId;
     }
 
     public ControlledShutdownRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.CONTROLLED_SHUTDOWN, version);
         brokerId = struct.getInt(BROKER_ID_KEY_NAME);
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
index a77a373..29ecd01 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
@@ -123,14 +123,14 @@ public class CreateAclsRequest extends AbstractRequest {
     private final List<AclCreation> aclCreations;
 
     CreateAclsRequest(short version, List<AclCreation> aclCreations) {
-        super(version);
+        super(ApiKeys.CREATE_ACLS, version);
         this.aclCreations = aclCreations;
 
         validate(aclCreations);
     }
 
     public CreateAclsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.CREATE_ACLS, version);
         this.aclCreations = new ArrayList<>();
         for (Object creationStructObj : struct.getArray(CREATIONS_KEY_NAME)) {
             Struct creationStruct = (Struct) creationStructObj;
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java
index 68e480f..3277f10 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java
@@ -53,13 +53,13 @@ public class CreateDelegationTokenRequest extends AbstractRequest {
     private final long maxLifeTime;
 
     private CreateDelegationTokenRequest(short version, List<KafkaPrincipal> renewers, long maxLifeTime) {
-        super(version);
+        super(ApiKeys.CREATE_DELEGATION_TOKEN, version);
         this.maxLifeTime = maxLifeTime;
         this.renewers = renewers;
     }
 
     public CreateDelegationTokenRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.CREATE_DELEGATION_TOKEN, version);
         maxLifeTime = struct.getLong(MAX_LIFE_TIME_KEY_NAME);
         Object[] renewerArray = struct.getArray(RENEWERS_KEY_NAME);
         renewers = new ArrayList<>();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java
index 5e776bb..795a66a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java
@@ -107,7 +107,7 @@ public class CreatePartitionsRequest extends AbstractRequest {
     }
 
     CreatePartitionsRequest(Map<String, NewPartitions> newPartitions, int timeout, boolean validateOnly, short apiVersion) {
-        super(apiVersion);
+        super(ApiKeys.CREATE_PARTITIONS, apiVersion);
         this.newPartitions = newPartitions;
         this.duplicates = Collections.emptySet();
         this.timeout = timeout;
@@ -115,7 +115,7 @@ public class CreatePartitionsRequest extends AbstractRequest {
     }
 
     public CreatePartitionsRequest(Struct struct, short apiVersion) {
-        super(apiVersion);
+        super(ApiKeys.CREATE_PARTITIONS, apiVersion);
         Object[] topicCountArray = struct.getArray(TOPIC_PARTITIONS_KEY_NAME);
         Map<String, NewPartitions> counts = new HashMap<>(topicCountArray.length);
         Set<String> dupes = new HashSet<>();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
index aa346f5..9f05c5a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
@@ -197,7 +197,7 @@ public class CreateTopicsRequest extends AbstractRequest {
     public static final short NO_REPLICATION_FACTOR = -1;
 
     private CreateTopicsRequest(Map<String, TopicDetails> topics, Integer timeout, boolean validateOnly, short version) {
-        super(version);
+        super(ApiKeys.CREATE_TOPICS, version);
         this.topics = topics;
         this.timeout = timeout;
         this.validateOnly = validateOnly;
@@ -205,7 +205,7 @@ public class CreateTopicsRequest extends AbstractRequest {
     }
 
     public CreateTopicsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.CREATE_TOPICS, version);
 
         Object[] requestStructs = struct.getArray(REQUESTS_KEY_NAME);
         Map<String, TopicDetails> topics = new HashMap<>();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
index 4c19a4a..c3fc194 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
@@ -96,14 +96,14 @@ public class DeleteAclsRequest extends AbstractRequest {
     private final List<AclBindingFilter> filters;
 
     DeleteAclsRequest(short version, List<AclBindingFilter> filters) {
-        super(version);
+        super(ApiKeys.DELETE_ACLS, version);
         this.filters = filters;
 
         validate(version, filters);
     }
 
     public DeleteAclsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DELETE_ACLS, version);
         this.filters = new ArrayList<>();
         for (Object filterStructObj : struct.getArray(FILTERS)) {
             Struct filterStruct = (Struct) filterStructObj;
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java
index 29604a5..f2a5d92 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java
@@ -74,12 +74,12 @@ public class DeleteGroupsRequest extends AbstractRequest {
     }
 
     private DeleteGroupsRequest(Set<String> groups, short version) {
-        super(version);
+        super(ApiKeys.DELETE_GROUPS, version);
         this.groups = groups;
     }
 
     public DeleteGroupsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DELETE_GROUPS, version);
         Object[] groupsArray = struct.getArray(GROUPS_KEY_NAME);
         Set<String> groups = new HashSet<>(groupsArray.length);
         for (Object group : groupsArray)
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
index ad3db60..7ea5553 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
@@ -104,7 +104,7 @@ public class DeleteRecordsRequest extends AbstractRequest {
 
 
     public DeleteRecordsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DELETE_RECORDS, version);
         partitionOffsets = new HashMap<>();
         for (Object topicStructObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicStruct = (Struct) topicStructObj;
@@ -120,14 +120,14 @@ public class DeleteRecordsRequest extends AbstractRequest {
     }
 
     public DeleteRecordsRequest(int timeout, Map<TopicPartition, Long> partitionOffsets, short version) {
-        super(version);
+        super(ApiKeys.DELETE_RECORDS, version);
         this.timeout = timeout;
         this.partitionOffsets = partitionOffsets;
     }
     @Override
     protected Struct toStruct() {
         Struct struct = new Struct(ApiKeys.DELETE_RECORDS.requestSchema(version()));
-        Map<String, Map<Integer, Long>> offsetsByTopic = CollectionUtils.groupDataByTopic(partitionOffsets);
+        Map<String, Map<Integer, Long>> offsetsByTopic = CollectionUtils.groupPartitionDataByTopic(partitionOffsets);
         struct.set(TIMEOUT_KEY_NAME, timeout);
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Long>> offsetsByTopicEntry : offsetsByTopic.entrySet()) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
index 0b494ba..311be1f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
@@ -136,7 +136,7 @@ public class DeleteRecordsResponse extends AbstractResponse {
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DELETE_RECORDS.responseSchema(version));
         struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
-        Map<String, Map<Integer, PartitionResponse>> responsesByTopic = CollectionUtils.groupDataByTopic(responses);
+        Map<String, Map<Integer, PartitionResponse>> responsesByTopic = CollectionUtils.groupPartitionDataByTopic(responses);
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionResponse>> responsesByTopicEntry : responsesByTopic.entrySet()) {
             Struct topicStruct = struct.instance(TOPICS_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
index 87f14b4..facb55e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
@@ -92,13 +92,13 @@ public class DeleteTopicsRequest extends AbstractRequest {
     }
 
     private DeleteTopicsRequest(Set<String> topics, Integer timeout, short version) {
-        super(version);
+        super(ApiKeys.DELETE_TOPICS, version);
         this.topics = topics;
         this.timeout = timeout;
     }
 
     public DeleteTopicsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DELETE_TOPICS, version);
         Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
         Set<String> topics = new HashSet<>(topicsArray.length);
         for (Object topic : topicsArray)
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
index d219839..04bfee8 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
@@ -86,14 +86,14 @@ public class DescribeAclsRequest extends AbstractRequest {
     private final AclBindingFilter filter;
 
     DescribeAclsRequest(AclBindingFilter filter, short version) {
-        super(version);
+        super(ApiKeys.DELETE_ACLS, version);
         this.filter = filter;
 
         validate(filter, version);
     }
 
     public DescribeAclsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DELETE_ACLS, version);
         ResourcePatternFilter resourceFilter = RequestUtils.resourcePatternFilterFromStructFields(struct);
         AccessControlEntryFilter entryFilter = RequestUtils.aceFilterFromStructFields(struct);
         this.filter = new AclBindingFilter(resourceFilter, entryFilter);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
index 781cd45..0ee256f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
@@ -100,13 +100,13 @@ public class DescribeConfigsRequest extends AbstractRequest {
     private final boolean includeSynonyms;
 
     public DescribeConfigsRequest(short version, Map<ConfigResource, Collection<String>> resourceToConfigNames, boolean includeSynonyms) {
-        super(version);
+        super(ApiKeys.DESCRIBE_CONFIGS, version);
         this.resourceToConfigNames = Objects.requireNonNull(resourceToConfigNames, "resourceToConfigNames");
         this.includeSynonyms = includeSynonyms;
     }
 
     public DescribeConfigsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DESCRIBE_CONFIGS, version);
         Object[] resourcesArray = struct.getArray(RESOURCES_KEY_NAME);
         resourceToConfigNames = new HashMap<>(resourcesArray.length);
         for (Object resourceObj : resourcesArray) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java
index 574bbcc..21e1460 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java
@@ -69,12 +69,12 @@ public class DescribeDelegationTokenRequest extends AbstractRequest {
     }
 
     private DescribeDelegationTokenRequest(short version, List<KafkaPrincipal> owners) {
-        super(version);
+        super(ApiKeys.DESCRIBE_DELEGATION_TOKEN, version);
         this.owners = owners;
     }
 
     public DescribeDelegationTokenRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.DESCRIBE_DELEGATION_TOKEN, versionId);
 
         Object[] ownerArray = struct.getArray(OWNER_KEY_NAME);
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
index 8ea4a8c..006af4f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
@@ -72,12 +72,12 @@ public class DescribeGroupsRequest extends AbstractRequest {
     private final List<String> groupIds;
 
     private DescribeGroupsRequest(List<String> groupIds, short version) {
-        super(version);
+        super(ApiKeys.DESCRIBE_GROUPS, version);
         this.groupIds = groupIds;
     }
 
     public DescribeGroupsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DESCRIBE_GROUPS, version);
         this.groupIds = new ArrayList<>();
         for (Object groupId : struct.getArray(GROUP_IDS_KEY_NAME))
             this.groupIds.add((String) groupId);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
index 3728991..e16cc18 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
@@ -86,7 +86,7 @@ public class DescribeLogDirsRequest extends AbstractRequest {
     }
 
     public DescribeLogDirsRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.DESCRIBE_LOG_DIRS, version);
 
         if (struct.getArray(TOPICS_KEY_NAME) == null) {
             topicPartitions = null;
@@ -105,7 +105,7 @@ public class DescribeLogDirsRequest extends AbstractRequest {
 
     // topicPartitions == null indicates requesting all partitions, and an empty list indicates requesting no partitions.
     public DescribeLogDirsRequest(Set<TopicPartition> topicPartitions, short version) {
-        super(version);
+        super(ApiKeys.DESCRIBE_LOG_DIRS, version);
         this.topicPartitions = topicPartitions;
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
index 41c2617..ee6d95c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
@@ -137,7 +137,7 @@ public class DescribeLogDirsResponse extends AbstractResponse {
             logDirStruct.set(ERROR_CODE, logDirInfo.error.code());
             logDirStruct.set(LOG_DIR_KEY_NAME, logDirInfosEntry.getKey());
 
-            Map<String, Map<Integer, ReplicaInfo>> replicaInfosByTopic = CollectionUtils.groupDataByTopic(logDirInfo.replicaInfos);
+            Map<String, Map<Integer, ReplicaInfo>> replicaInfosByTopic = CollectionUtils.groupPartitionDataByTopic(logDirInfo.replicaInfos);
             List<Struct> topicStructArray = new ArrayList<>();
             for (Map.Entry<String, Map<Integer, ReplicaInfo>> replicaInfosByTopicEntry : replicaInfosByTopic.entrySet()) {
                 Struct topicStruct = logDirStruct.instance(TOPICS_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
index 0ec22dc..833a7fe 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
@@ -89,7 +89,7 @@ public class EndTxnRequest extends AbstractRequest {
     private final TransactionResult result;
 
     private EndTxnRequest(short version, String transactionalId, long producerId, short producerEpoch, TransactionResult result) {
-        super(version);
+        super(ApiKeys.END_TXN, version);
         this.transactionalId = transactionalId;
         this.producerId = producerId;
         this.producerEpoch = producerEpoch;
@@ -97,7 +97,7 @@ public class EndTxnRequest extends AbstractRequest {
     }
 
     public EndTxnRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.END_TXN, version);
         this.transactionalId = struct.get(TRANSACTIONAL_ID);
         this.producerId = struct.get(PRODUCER_ID);
         this.producerEpoch = struct.get(PRODUCER_EPOCH);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java b/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java
index ce938aa..06dfef9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java
@@ -25,7 +25,6 @@ import java.util.Objects;
 /**
  * The offset, fetched from a leader, for a particular partition.
  */
-
 public class EpochEndOffset {
     public static final long UNDEFINED_EPOCH_OFFSET = NO_PARTITION_LEADER_EPOCH;
     public static final int UNDEFINED_EPOCH = NO_PARTITION_LEADER_EPOCH;
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java
index 21edb5e..5b99676 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java
@@ -44,14 +44,14 @@ public class ExpireDelegationTokenRequest extends AbstractRequest {
     private static final Schema TOKEN_EXPIRE_REQUEST_V1 = TOKEN_EXPIRE_REQUEST_V0;
 
     private ExpireDelegationTokenRequest(short version, ByteBuffer hmac, long renewTimePeriod) {
-        super(version);
+        super(ApiKeys.EXPIRE_DELEGATION_TOKEN, version);
 
         this.hmac = hmac;
         this.expiryTimePeriod = renewTimePeriod;
     }
 
     public ExpireDelegationTokenRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.EXPIRE_DELEGATION_TOKEN, versionId);
 
         hmac = struct.getBytes(HMAC_KEY_NAME);
         expiryTimePeriod = struct.getLong(EXPIRY_TIME_PERIOD_KEY_NAME);
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 e013f5e..32eb24d 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
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -36,154 +35,166 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 
+import static org.apache.kafka.common.protocol.CommonFields.CURRENT_LEADER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.INT32;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
-import static org.apache.kafka.common.protocol.types.Type.INT8;
 import static org.apache.kafka.common.requests.FetchMetadata.FINAL_EPOCH;
 import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
 
 public class FetchRequest extends AbstractRequest {
     public static final int CONSUMER_REPLICA_ID = -1;
-    private static final String REPLICA_ID_KEY_NAME = "replica_id";
-    private static final String MAX_WAIT_KEY_NAME = "max_wait_time";
-    private static final String MIN_BYTES_KEY_NAME = "min_bytes";
-    private static final String ISOLATION_LEVEL_KEY_NAME = "isolation_level";
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String FORGOTTEN_TOPICS_DATA = "forgotten_topics_data";
 
-    // request and partition level name
-    private static final String MAX_BYTES_KEY_NAME = "max_bytes";
-
-    // topic level field names
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-
-    // partition level field names
-    private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset";
-    private static final String LOG_START_OFFSET_KEY_NAME = "log_start_offset";
-
-    private static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(
-            PARTITION_ID,
-            new Field(FETCH_OFFSET_KEY_NAME, INT64, "Message offset."),
-            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to fetch."));
-
-    // FETCH_REQUEST_PARTITION_V5 added log_start_offset field - the earliest available offset of partition data that can be consumed.
-    private static final Schema FETCH_REQUEST_PARTITION_V5 = new Schema(
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "Topics to fetch in the order provided.");
+    private static final Field.ComplexArray FORGOTTEN_TOPICS = new Field.ComplexArray("forgotten_topics_data",
+            "Topics to remove from the fetch session.");
+    private static final Field.Int32 MAX_BYTES = new Field.Int32("max_bytes",
+            "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
+                    "if the first message in the first non-empty partition of the fetch is larger than this " +
+                    "value, the message will still be returned to ensure that progress can be made.");
+    private static final Field.Int8 ISOLATION_LEVEL = new Field.Int8("isolation_level",
+            "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
+                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
+                    "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
+                    "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
+                    "and enables the inclusion of the list of aborted transactions in the result, which allows " +
+                    "consumers to discard ABORTED transactional records");
+    private static final Field.Int32 SESSION_ID = new Field.Int32("session_id", "The fetch session ID");
+    private static final Field.Int32 SESSION_EPOCH = new Field.Int32("session_epoch", "The fetch session epoch");
+
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "Partitions to fetch.");
+
+    // partition level fields
+    private static final Field.Int32 REPLICA_ID = new Field.Int32("replica_id",
+            "Broker id of the follower. For normal consumers, use -1.");
+    private static final Field.Int64 FETCH_OFFSET = new Field.Int64("fetch_offset", "Message offset.");
+    private static final Field.Int32 PARTITION_MAX_BYTES = new Field.Int32("partition_max_bytes",
+            "Maximum bytes to fetch.");
+    private static final Field.Int32 MAX_WAIT_TIME = new Field.Int32("max_wait_time",
+            "Maximum time in ms to wait for the response.");
+    private static final Field.Int32 MIN_BYTES = new Field.Int32("min_bytes",
+            "Minimum bytes to accumulate in the response.");
+    private static final Field.Int64 LOG_START_OFFSET = new Field.Int64("log_start_offset",
+            "Earliest available offset of the follower replica. " +
+                    "The field is only used when request is sent by follower. ");
+
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
-            new Field(FETCH_OFFSET_KEY_NAME, INT64, "Message offset."),
-            new Field(LOG_START_OFFSET_KEY_NAME, INT64, "Earliest available offset of the follower replica. " +
-                            "The field is only used when request is sent by follower. "),
-            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to fetch."));
+            FETCH_OFFSET,
+            PARTITION_MAX_BYTES);
 
-    private static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_REQUEST_PARTITION_V0), "Partitions to fetch."));
-
-    private static final Schema FETCH_REQUEST_TOPIC_V5 = new Schema(
-            TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_REQUEST_PARTITION_V5), "Partitions to fetch."));
+            PARTITIONS_V0);
 
     private static final Schema FETCH_REQUEST_V0 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
-            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch."));
+            REPLICA_ID,
+            MAX_WAIT_TIME,
+            MIN_BYTES,
+            TOPICS_V0);
 
     // The V1 Fetch Request body is the same as V0.
     // Only the version number is incremented to indicate a newer client
     private static final Schema FETCH_REQUEST_V1 = FETCH_REQUEST_V0;
-    // The V2 Fetch Request body is the same as V1.
-    // Only the version number is incremented to indicate the client support message format V1 which uses
-    // relative offset and has timestamp.
+
+    // V2 bumped to indicate the client support message format V1 which uses relative offset and has timestamp.
     private static final Schema FETCH_REQUEST_V2 = FETCH_REQUEST_V1;
-    // Fetch Request V3 added top level max_bytes field - the total size of partition data to accumulate in response.
+
+    // V3 added top level max_bytes field - the total size of partition data to accumulate in response.
     // The partition ordering is now relevant - partitions will be processed in order they appear in request.
     private static final Schema FETCH_REQUEST_V3 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
-            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
-            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-                    "if the first message in the first non-empty partition of the fetch is larger than this " +
-                    "value, the message will still be returned to ensure that progress can be made."),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch in the order provided."));
+            REPLICA_ID,
+            MAX_WAIT_TIME,
+            MIN_BYTES,
+            MAX_BYTES,
+            TOPICS_V0);
+
+    // V4 adds the fetch isolation level and exposes magic v2 (via the response).
 
-    // The V4 Fetch Request adds the fetch isolation level and exposes magic v2 (via the response).
     private static final Schema FETCH_REQUEST_V4 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
-            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
-            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-                    "if the first message in the first non-empty partition of the fetch is larger than this " +
-                    "value, the message will still be returned to ensure that progress can be made."),
-            new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
-                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
-                    "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
-                    "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
-                    "and enables the inclusion of the list of aborted transactions in the result, which allows " +
-                    "consumers to discard ABORTED transactional records"),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch in the order provided."));
+            REPLICA_ID,
+            MAX_WAIT_TIME,
+            MIN_BYTES,
+            MAX_BYTES,
+            ISOLATION_LEVEL,
+            TOPICS_V0);
+
+
+    // V5 added log_start_offset field - the earliest available offset of partition data that can be consumed.
+    private static final Field PARTITIONS_V5 = PARTITIONS.withFields(
+            PARTITION_ID,
+            FETCH_OFFSET,
+            LOG_START_OFFSET,
+            PARTITION_MAX_BYTES);
+
+    private static final Field TOPICS_V5 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V5);
 
-    // FETCH_REQUEST_V5 added a per-partition log_start_offset field - the earliest available offset of partition data that can be consumed.
     private static final Schema FETCH_REQUEST_V5 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
-            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
-            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-                    "if the first message in the first non-empty partition of the fetch is larger than this " +
-                    "value, the message will still be returned to ensure that progress can be made."),
-            new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
-                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
-                    "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
-                    "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
-                    "and enables the inclusion of the list of aborted transactions in the result, which allows " +
-                    "consumers to discard ABORTED transactional records"),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V5), "Topics to fetch in the order provided."));
-
-    /**
-     * The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5.
-     * The version number is bumped up to indicate that the client supports KafkaStorageException.
-     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
-     */
+            REPLICA_ID,
+            MAX_WAIT_TIME,
+            MIN_BYTES,
+            MAX_BYTES,
+            ISOLATION_LEVEL,
+            TOPICS_V5);
+
+    // V6 bumped up to indicate that the client supports KafkaStorageException. The KafkaStorageException will be
+    // translated to NotLeaderForPartitionException in the response if version <= 5
     private static final Schema FETCH_REQUEST_V6 = FETCH_REQUEST_V5;
 
-    // FETCH_REQUEST_V7 added incremental fetch requests.
-    public static final Field.Int32 SESSION_ID = new Field.Int32("session_id", "The fetch session ID");
-    public static final Field.Int32 EPOCH = new Field.Int32("epoch", "The fetch epoch");
-
-    private static final Schema FORGOTTEN_TOPIC_DATA = new Schema(
-        TOPIC_NAME,
-        new Field(PARTITIONS_KEY_NAME, new ArrayOf(Type.INT32),
-            "Partitions to remove from the fetch session."));
+    // V7 added incremental fetch requests.
+    private static final Field.Array FORGOTTEN_PARTITIONS = new Field.Array("partitions", Type.INT32,
+            "Partitions to remove from the fetch session.");
+    private static final Field FORGOTTEN_TOPIC_DATA_V7 = FORGOTTEN_TOPICS.withFields(
+            TOPIC_NAME,
+            FORGOTTEN_PARTITIONS);
 
     private static final Schema FETCH_REQUEST_V7 = new Schema(
-        new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-        new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
-        new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
-        new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-            "if the first message in the first non-empty partition of the fetch is larger than this " +
-            "value, the message will still be returned to ensure that progress can be made."),
-        new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
-            "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
-            "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
-            "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
-            "and enables the inclusion of the list of aborted transactions in the result, which allows " +
-            "consumers to discard ABORTED transactional records"),
-        SESSION_ID,
-        EPOCH,
-        new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V5), "Topics to fetch in the order provided."),
-        new Field(FORGOTTEN_TOPICS_DATA, new ArrayOf(FORGOTTEN_TOPIC_DATA), "Topics to remove from the fetch session."));
-
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+            REPLICA_ID,
+            MAX_WAIT_TIME,
+            MIN_BYTES,
+            MAX_BYTES,
+            ISOLATION_LEVEL,
+            SESSION_ID,
+            SESSION_EPOCH,
+            TOPICS_V5,
+            FORGOTTEN_TOPIC_DATA_V7);
+
+    // V8 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema FETCH_REQUEST_V8 = FETCH_REQUEST_V7;
 
+    // V9 adds the current leader epoch (see KIP-320)
+    private static final Field FETCH_REQUEST_PARTITION_V9 = PARTITIONS.withFields(
+            PARTITION_ID,
+            CURRENT_LEADER_EPOCH,
+            FETCH_OFFSET,
+            LOG_START_OFFSET,
+            PARTITION_MAX_BYTES);
+
+    private static final Field FETCH_REQUEST_TOPIC_V9 = TOPICS.withFields(
+            TOPIC_NAME,
+            FETCH_REQUEST_PARTITION_V9);
+
+    private static final Schema FETCH_REQUEST_V9 = new Schema(
+            REPLICA_ID,
+            MAX_WAIT_TIME,
+            MIN_BYTES,
+            MAX_BYTES,
+            ISOLATION_LEVEL,
+            SESSION_ID,
+            SESSION_EPOCH,
+            FETCH_REQUEST_TOPIC_V9,
+            FORGOTTEN_TOPIC_DATA_V7);
+
     public static Schema[] schemaVersions() {
         return new Schema[]{FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2, FETCH_REQUEST_V3, FETCH_REQUEST_V4,
-            FETCH_REQUEST_V5, FETCH_REQUEST_V6, FETCH_REQUEST_V7, FETCH_REQUEST_V8};
-    };
+            FETCH_REQUEST_V5, FETCH_REQUEST_V6, FETCH_REQUEST_V7, FETCH_REQUEST_V8, FETCH_REQUEST_V9};
+    }
 
     // default values for older versions where a request level limit did not exist
     public static final int DEFAULT_RESPONSE_MAX_BYTES = Integer.MAX_VALUE;
@@ -207,21 +218,27 @@ public class FetchRequest extends AbstractRequest {
         public final long fetchOffset;
         public final long logStartOffset;
         public final int maxBytes;
+        public final Optional<Integer> currentLeaderEpoch;
 
-        public PartitionData(long fetchOffset, long logStartOffset, int maxBytes) {
+        public PartitionData(long fetchOffset, long logStartOffset, int maxBytes, Optional<Integer> currentLeaderEpoch) {
             this.fetchOffset = fetchOffset;
             this.logStartOffset = logStartOffset;
             this.maxBytes = maxBytes;
+            this.currentLeaderEpoch = currentLeaderEpoch;
         }
 
         @Override
         public String toString() {
-            return "(offset=" + fetchOffset + ", logStartOffset=" + logStartOffset + ", maxBytes=" + maxBytes + ")";
+            return "(offset=" + fetchOffset +
+                    ", logStartOffset=" + logStartOffset +
+                    ", maxBytes=" + maxBytes +
+                    ", currentLeaderEpoch=" + currentLeaderEpoch +
+                    ")";
         }
 
         @Override
         public int hashCode() {
-            return Objects.hash(fetchOffset, logStartOffset, maxBytes);
+            return Objects.hash(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch);
         }
 
         @Override
@@ -231,7 +248,8 @@ public class FetchRequest extends AbstractRequest {
             PartitionData that = (PartitionData) o;
             return Objects.equals(fetchOffset, that.fetchOffset) &&
                 Objects.equals(logStartOffset, that.logStartOffset) &&
-                Objects.equals(maxBytes, that.maxBytes);
+                Objects.equals(maxBytes, that.maxBytes) &&
+                Objects.equals(currentLeaderEpoch, that.currentLeaderEpoch);
         }
     }
 
@@ -346,7 +364,7 @@ public class FetchRequest extends AbstractRequest {
     private FetchRequest(short version, int replicaId, int maxWait, int minBytes, int maxBytes,
                          Map<TopicPartition, PartitionData> fetchData, IsolationLevel isolationLevel,
                          List<TopicPartition> toForget, FetchMetadata metadata) {
-        super(version);
+        super(ApiKeys.FETCH, version);
         this.replicaId = replicaId;
         this.maxWait = maxWait;
         this.minBytes = minBytes;
@@ -358,44 +376,44 @@ public class FetchRequest extends AbstractRequest {
     }
 
     public FetchRequest(Struct struct, short version) {
-        super(version);
-        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
-        maxWait = struct.getInt(MAX_WAIT_KEY_NAME);
-        minBytes = struct.getInt(MIN_BYTES_KEY_NAME);
-        if (struct.hasField(MAX_BYTES_KEY_NAME))
-            maxBytes = struct.getInt(MAX_BYTES_KEY_NAME);
-        else
-            maxBytes = DEFAULT_RESPONSE_MAX_BYTES;
-        if (struct.hasField(ISOLATION_LEVEL_KEY_NAME))
-            isolationLevel = IsolationLevel.forId(struct.getByte(ISOLATION_LEVEL_KEY_NAME));
+        super(ApiKeys.FETCH, version);
+        replicaId = struct.get(REPLICA_ID);
+        maxWait = struct.get(MAX_WAIT_TIME);
+        minBytes = struct.get(MIN_BYTES);
+        maxBytes = struct.getOrElse(MAX_BYTES, DEFAULT_RESPONSE_MAX_BYTES);
+
+        if (struct.hasField(ISOLATION_LEVEL))
+            isolationLevel = IsolationLevel.forId(struct.get(ISOLATION_LEVEL));
         else
             isolationLevel = IsolationLevel.READ_UNCOMMITTED;
         toForget = new ArrayList<>(0);
-        if (struct.hasField(FORGOTTEN_TOPICS_DATA)) {
-            for (Object forgottenTopicObj : struct.getArray(FORGOTTEN_TOPICS_DATA)) {
+        if (struct.hasField(FORGOTTEN_TOPICS)) {
+            for (Object forgottenTopicObj : struct.get(FORGOTTEN_TOPICS)) {
                 Struct forgottenTopic = (Struct) forgottenTopicObj;
                 String topicName = forgottenTopic.get(TOPIC_NAME);
-                for (Object partObj : forgottenTopic.getArray(PARTITIONS_KEY_NAME)) {
+                for (Object partObj : forgottenTopic.get(FORGOTTEN_PARTITIONS)) {
                     Integer part = (Integer) partObj;
                     toForget.add(new TopicPartition(topicName, part));
                 }
             }
         }
         metadata = new FetchMetadata(struct.getOrElse(SESSION_ID, INVALID_SESSION_ID),
-            struct.getOrElse(EPOCH, FINAL_EPOCH));
+            struct.getOrElse(SESSION_EPOCH, FINAL_EPOCH));
 
         fetchData = new LinkedHashMap<>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+        for (Object topicResponseObj : struct.get(TOPICS)) {
             Struct topicResponse = (Struct) topicResponseObj;
             String topic = topicResponse.get(TOPIC_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionResponseObj : topicResponse.get(PARTITIONS)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
                 int partition = partitionResponse.get(PARTITION_ID);
-                long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
-                int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
-                long logStartOffset = partitionResponse.hasField(LOG_START_OFFSET_KEY_NAME) ?
-                    partitionResponse.getLong(LOG_START_OFFSET_KEY_NAME) : INVALID_LOG_START_OFFSET;
-                PartitionData partitionData = new PartitionData(offset, logStartOffset, maxBytes);
+                long offset = partitionResponse.get(FETCH_OFFSET);
+                int maxBytes = partitionResponse.get(PARTITION_MAX_BYTES);
+                long logStartOffset = partitionResponse.getOrElse(LOG_START_OFFSET, INVALID_LOG_START_OFFSET);
+
+                // Current leader epoch added in v9
+                Optional<Integer> currentLeaderEpoch = RequestUtils.getLeaderEpoch(partitionResponse, CURRENT_LEADER_EPOCH);
+                PartitionData partitionData = new PartitionData(offset, logStartOffset, maxBytes, currentLeaderEpoch);
                 fetchData.put(new TopicPartition(topic, partition), partitionData);
             }
         }
@@ -410,14 +428,14 @@ public class FetchRequest extends AbstractRequest {
         // may not be any partitions at all in the response.  For this reason, the top-level error code
         // is essential for them.
         Errors error = Errors.forException(e);
-        LinkedHashMap<TopicPartition, FetchResponse.PartitionData> responseData = new LinkedHashMap<>();
+        LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>();
         for (Map.Entry<TopicPartition, PartitionData> entry : fetchData.entrySet()) {
-            FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(error,
+            FetchResponse.PartitionData<MemoryRecords> partitionResponse = new FetchResponse.PartitionData<>(error,
                 FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET,
                 FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY);
             responseData.put(entry.getKey(), partitionResponse);
         }
-        return new FetchResponse(error, responseData, throttleTimeMs, metadata.sessionId());
+        return new FetchResponse<>(error, responseData, throttleTimeMs, metadata.sessionId());
     }
 
     public int replicaId() {
@@ -466,53 +484,47 @@ public class FetchRequest extends AbstractRequest {
         List<TopicAndPartitionData<PartitionData>> topicsData =
             TopicAndPartitionData.batchByTopic(fetchData.entrySet().iterator());
 
-        struct.set(REPLICA_ID_KEY_NAME, replicaId);
-        struct.set(MAX_WAIT_KEY_NAME, maxWait);
-        struct.set(MIN_BYTES_KEY_NAME, minBytes);
-        if (struct.hasField(MAX_BYTES_KEY_NAME))
-            struct.set(MAX_BYTES_KEY_NAME, maxBytes);
-        if (struct.hasField(ISOLATION_LEVEL_KEY_NAME))
-            struct.set(ISOLATION_LEVEL_KEY_NAME, isolationLevel.id());
+        struct.set(REPLICA_ID, replicaId);
+        struct.set(MAX_WAIT_TIME, maxWait);
+        struct.set(MIN_BYTES, minBytes);
+        struct.setIfExists(MAX_BYTES, maxBytes);
+        struct.setIfExists(ISOLATION_LEVEL, isolationLevel.id());
         struct.setIfExists(SESSION_ID, metadata.sessionId());
-        struct.setIfExists(EPOCH, metadata.epoch());
+        struct.setIfExists(SESSION_EPOCH, metadata.epoch());
 
         List<Struct> topicArray = new ArrayList<>();
         for (TopicAndPartitionData<PartitionData> topicEntry : topicsData) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            Struct topicData = struct.instance(TOPICS);
             topicData.set(TOPIC_NAME, topicEntry.topic);
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.partitions.entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                Struct partitionData = topicData.instance(PARTITIONS);
                 partitionData.set(PARTITION_ID, partitionEntry.getKey());
-                partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.fetchOffset);
-                if (partitionData.hasField(LOG_START_OFFSET_KEY_NAME))
-                    partitionData.set(LOG_START_OFFSET_KEY_NAME, fetchPartitionData.logStartOffset);
-                partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes);
+                partitionData.set(FETCH_OFFSET, fetchPartitionData.fetchOffset);
+                partitionData.set(PARTITION_MAX_BYTES, fetchPartitionData.maxBytes);
+                partitionData.setIfExists(LOG_START_OFFSET, fetchPartitionData.logStartOffset);
+                RequestUtils.setLeaderEpochIfExists(partitionData, CURRENT_LEADER_EPOCH, fetchPartitionData.currentLeaderEpoch);
                 partitionArray.add(partitionData);
             }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicData.set(PARTITIONS, partitionArray.toArray());
             topicArray.add(topicData);
         }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
-        if (struct.hasField(FORGOTTEN_TOPICS_DATA)) {
+        struct.set(TOPICS, topicArray.toArray());
+        if (struct.hasField(FORGOTTEN_TOPICS)) {
             Map<String, List<Integer>> topicsToPartitions = new HashMap<>();
             for (TopicPartition part : toForget) {
-                List<Integer> partitions = topicsToPartitions.get(part.topic());
-                if (partitions == null) {
-                    partitions = new ArrayList<>();
-                    topicsToPartitions.put(part.topic(), partitions);
-                }
+                List<Integer> partitions = topicsToPartitions.computeIfAbsent(part.topic(), topic -> new ArrayList<>());
                 partitions.add(part.partition());
             }
             List<Struct> toForgetStructs = new ArrayList<>();
             for (Map.Entry<String, List<Integer>> entry : topicsToPartitions.entrySet()) {
-                Struct toForgetStruct = struct.instance(FORGOTTEN_TOPICS_DATA);
+                Struct toForgetStruct = struct.instance(FORGOTTEN_TOPICS);
                 toForgetStruct.set(TOPIC_NAME, entry.getKey());
-                toForgetStruct.set(PARTITIONS_KEY_NAME, entry.getValue().toArray());
+                toForgetStruct.set(FORGOTTEN_PARTITIONS, entry.getValue().toArray());
                 toForgetStructs.add(toForgetStruct);
             }
-            struct.set(FORGOTTEN_TOPICS_DATA, toForgetStructs.toArray());
+            struct.set(FORGOTTEN_TOPICS, toForgetStructs.toArray());
         }
         return struct;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
index 16e3396..2e0eaf2 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
@@ -18,7 +18,6 @@ package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.network.ByteBufferSend;
-import org.apache.kafka.common.record.MultiRecordsSend;
 import org.apache.kafka.common.network.Send;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
@@ -28,6 +27,7 @@ import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.BaseRecords;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MultiRecordsSend;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
@@ -43,13 +43,20 @@ import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
 import static org.apache.kafka.common.protocol.types.Type.RECORDS;
 import static org.apache.kafka.common.protocol.types.Type.STRING;
 import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
 
 /**
  * This wrapper supports all versions of the Fetch API
+ *
+ * Possible error codes:
+ *
+ *  OFFSET_OUT_OF_RANGE (1)
+ *  UNKNOWN_TOPIC_OR_PARTITION (3)
+ *  NOT_LEADER_FOR_PARTITION (6)
+ *  REPLICA_NOT_AVAILABLE (9)
+ *  UNKNOWN (-1)
  */
 public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
 
@@ -58,22 +65,20 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
     // topic level field names
     private static final String PARTITIONS_KEY_NAME = "partition_responses";
 
-    // partition level field names
+    // partition level fields
+    private static final Field.Int64 HIGH_WATERMARK = new Field.Int64("high_watermark",
+            "Last committed offset.");
+    private static final Field.Int64 LOG_START_OFFSET = new Field.Int64("log_start_offset",
+            "Earliest available offset.");
+
     private static final String PARTITION_HEADER_KEY_NAME = "partition_header";
-    private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark";
-    private static final String LAST_STABLE_OFFSET_KEY_NAME = "last_stable_offset";
-    private static final String LOG_START_OFFSET_KEY_NAME = "log_start_offset";
     private static final String ABORTED_TRANSACTIONS_KEY_NAME = "aborted_transactions";
     private static final String RECORD_SET_KEY_NAME = "record_set";
 
-    // aborted transaction field names
-    private static final String PRODUCER_ID_KEY_NAME = "producer_id";
-    private static final String FIRST_OFFSET_KEY_NAME = "first_offset";
-
     private static final Schema FETCH_RESPONSE_PARTITION_HEADER_V0 = new Schema(
             PARTITION_ID,
             ERROR_CODE,
-            new Field(HIGH_WATERMARK_KEY_NAME, INT64, "Last committed offset."));
+            HIGH_WATERMARK);
     private static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(
             new Field(PARTITION_HEADER_KEY_NAME, FETCH_RESPONSE_PARTITION_HEADER_V0),
             new Field(RECORD_SET_KEY_NAME, RECORDS));
@@ -85,42 +90,47 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
     private static final Schema FETCH_RESPONSE_V0 = new Schema(
             new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
 
+    // V1 bumped for the addition of the throttle time
     private static final Schema FETCH_RESPONSE_V1 = new Schema(
             THROTTLE_TIME_MS,
             new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
-    // Even though fetch response v2 has the same protocol as v1, the record set in the response is different. In v1,
-    // record set only includes messages of v0 (magic byte 0). In v2, record set can include messages of v0 and v1
-    // (magic byte 0 and 1). For details, see Records, RecordBatch and Record.
+
+    // V2 bumped to indicate the client support message format V1 which uses relative offset and has timestamp.
     private static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1;
 
-    // The partition ordering is now relevant - partitions will be processed in order they appear in request.
+    // V3 bumped for addition of top-levl max_bytes field and to indicate that partition ordering is relevant
     private static final Schema FETCH_RESPONSE_V3 = FETCH_RESPONSE_V2;
 
-    // The v4 Fetch Response adds features for transactional consumption (the aborted transaction list and the
+    // V4 adds features for transactional consumption (the aborted transaction list and the
     // last stable offset). It also exposes messages with magic v2 (along with older formats).
-    private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V4 = new Schema(
-            new Field(PRODUCER_ID_KEY_NAME, INT64, "The producer id associated with the aborted transactions"),
-            new Field(FIRST_OFFSET_KEY_NAME, INT64, "The first offset in the aborted transaction"));
+    // aborted transaction field names
+    private static final Field.Int64 LAST_STABLE_OFFSET = new Field.Int64("last_stable_offset",
+            "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
+                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)");
+    private static final Field.Int64 PRODUCER_ID = new Field.Int64("producer_id",
+            "The producer id associated with the aborted transactions");
+    private static final Field.Int64 FIRST_OFFSET = new Field.Int64("first_offset",
+            "The first offset in the aborted transaction");
 
-    private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V5 = FETCH_RESPONSE_ABORTED_TRANSACTION_V4;
+    private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V4 = new Schema(
+            PRODUCER_ID,
+            FIRST_OFFSET);
 
     private static final Schema FETCH_RESPONSE_PARTITION_HEADER_V4 = new Schema(
             PARTITION_ID,
             ERROR_CODE,
-            new Field(HIGH_WATERMARK_KEY_NAME, INT64, "Last committed offset."),
-            new Field(LAST_STABLE_OFFSET_KEY_NAME, INT64, "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
-                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)"),
+            HIGH_WATERMARK,
+            LAST_STABLE_OFFSET,
             new Field(ABORTED_TRANSACTIONS_KEY_NAME, ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V4)));
 
-    // FETCH_RESPONSE_PARTITION_HEADER_V5 added log_start_offset field - the earliest available offset of partition data that can be consumed.
+    // V5 added log_start_offset field - the earliest available offset of partition data that can be consumed.
     private static final Schema FETCH_RESPONSE_PARTITION_HEADER_V5 = new Schema(
             PARTITION_ID,
             ERROR_CODE,
-            new Field(HIGH_WATERMARK_KEY_NAME, INT64, "Last committed offset."),
-            new Field(LAST_STABLE_OFFSET_KEY_NAME, INT64, "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
-                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)"),
-            new Field(LOG_START_OFFSET_KEY_NAME, INT64, "Earliest available offset."),
-            new Field(ABORTED_TRANSACTIONS_KEY_NAME, ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V5)));
+            HIGH_WATERMARK,
+            LAST_STABLE_OFFSET,
+            LOG_START_OFFSET,
+            new Field(ABORTED_TRANSACTIONS_KEY_NAME, ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V4)));
 
     private static final Schema FETCH_RESPONSE_PARTITION_V4 = new Schema(
             new Field(PARTITION_HEADER_KEY_NAME, FETCH_RESPONSE_PARTITION_HEADER_V4),
@@ -146,15 +156,12 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
             THROTTLE_TIME_MS,
             new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V5)));
 
-    /**
-     * The body of FETCH_RESPONSE_V6 is the same as FETCH_RESPONSE_V5.
-     * The version number is bumped up to indicate that the client supports KafkaStorageException.
-     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
-     */
+    // V6 bumped up to indicate that the client supports KafkaStorageException. The KafkaStorageException will
+    // be translated to NotLeaderForPartitionException in the response if version <= 5
     private static final Schema FETCH_RESPONSE_V6 = FETCH_RESPONSE_V5;
 
-    // FETCH_RESPONSE_V7 added incremental fetch responses and a top-level error code.
-    public static final Field.Int32 SESSION_ID = new Field.Int32("session_id", "The fetch session ID");
+    // V7 added incremental fetch responses and a top-level error code.
+    private static final Field.Int32 SESSION_ID = new Field.Int32("session_id", "The fetch session ID");
 
     private static final Schema FETCH_RESPONSE_V7 = new Schema(
             THROTTLE_TIME_MS,
@@ -162,32 +169,22 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
             SESSION_ID,
             new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V5)));
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V8 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema FETCH_RESPONSE_V8 = FETCH_RESPONSE_V7;
 
+    // V9 adds the current leader epoch (see KIP-320)
+    private static final Schema FETCH_RESPONSE_V9 = FETCH_RESPONSE_V8;
+
     public static Schema[] schemaVersions() {
         return new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2,
             FETCH_RESPONSE_V3, FETCH_RESPONSE_V4, FETCH_RESPONSE_V5, FETCH_RESPONSE_V6,
-            FETCH_RESPONSE_V7, FETCH_RESPONSE_V8};
+            FETCH_RESPONSE_V7, FETCH_RESPONSE_V8, FETCH_RESPONSE_V9};
     }
 
-
     public static final long INVALID_HIGHWATERMARK = -1L;
     public static final long INVALID_LAST_STABLE_OFFSET = -1L;
     public static final long INVALID_LOG_START_OFFSET = -1L;
 
-    /**
-     * Possible error codes:
-     *
-     *  OFFSET_OUT_OF_RANGE (1)
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)
-     *  NOT_LEADER_FOR_PARTITION (6)
-     *  REPLICA_NOT_AVAILABLE (9)
-     *  UNKNOWN (-1)
-     */
-
     private final int throttleTimeMs;
     private final Errors error;
     private final int sessionId;
@@ -318,13 +315,9 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
                 Struct partitionResponseHeader = partitionResponse.getStruct(PARTITION_HEADER_KEY_NAME);
                 int partition = partitionResponseHeader.get(PARTITION_ID);
                 Errors error = Errors.forCode(partitionResponseHeader.get(ERROR_CODE));
-                long highWatermark = partitionResponseHeader.getLong(HIGH_WATERMARK_KEY_NAME);
-                long lastStableOffset = INVALID_LAST_STABLE_OFFSET;
-                if (partitionResponseHeader.hasField(LAST_STABLE_OFFSET_KEY_NAME))
-                    lastStableOffset = partitionResponseHeader.getLong(LAST_STABLE_OFFSET_KEY_NAME);
-                long logStartOffset = INVALID_LOG_START_OFFSET;
-                if (partitionResponseHeader.hasField(LOG_START_OFFSET_KEY_NAME))
-                    logStartOffset = partitionResponseHeader.getLong(LOG_START_OFFSET_KEY_NAME);
+                long highWatermark = partitionResponseHeader.get(HIGH_WATERMARK);
+                long lastStableOffset = partitionResponseHeader.getOrElse(LAST_STABLE_OFFSET, INVALID_LAST_STABLE_OFFSET);
+                long logStartOffset = partitionResponseHeader.getOrElse(LOG_START_OFFSET, INVALID_LOG_START_OFFSET);
 
                 BaseRecords baseRecords = partitionResponse.getRecords(RECORD_SET_KEY_NAME);
                 if (!(baseRecords instanceof MemoryRecords))
@@ -338,8 +331,8 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
                         abortedTransactions = new ArrayList<>(abortedTransactionsArray.length);
                         for (Object abortedTransactionObj : abortedTransactionsArray) {
                             Struct abortedTransactionStruct = (Struct) abortedTransactionObj;
-                            long producerId = abortedTransactionStruct.getLong(PRODUCER_ID_KEY_NAME);
-                            long firstOffset = abortedTransactionStruct.getLong(FIRST_OFFSET_KEY_NAME);
+                            long producerId = abortedTransactionStruct.get(PRODUCER_ID);
+                            long firstOffset = abortedTransactionStruct.get(FIRST_OFFSET);
                             abortedTransactions.add(new AbortedTransaction(producerId, firstOffset));
                         }
                     }
@@ -490,10 +483,10 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
                 Struct partitionDataHeader = partitionData.instance(PARTITION_HEADER_KEY_NAME);
                 partitionDataHeader.set(PARTITION_ID, partitionEntry.getKey());
                 partitionDataHeader.set(ERROR_CODE, errorCode);
-                partitionDataHeader.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
+                partitionDataHeader.set(HIGH_WATERMARK, fetchPartitionData.highWatermark);
 
-                if (partitionDataHeader.hasField(LAST_STABLE_OFFSET_KEY_NAME)) {
-                    partitionDataHeader.set(LAST_STABLE_OFFSET_KEY_NAME, fetchPartitionData.lastStableOffset);
+                if (partitionDataHeader.hasField(LAST_STABLE_OFFSET)) {
+                    partitionDataHeader.set(LAST_STABLE_OFFSET, fetchPartitionData.lastStableOffset);
 
                     if (fetchPartitionData.abortedTransactions == null) {
                         partitionDataHeader.set(ABORTED_TRANSACTIONS_KEY_NAME, null);
@@ -501,16 +494,14 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
                         List<Struct> abortedTransactionStructs = new ArrayList<>(fetchPartitionData.abortedTransactions.size());
                         for (AbortedTransaction abortedTransaction : fetchPartitionData.abortedTransactions) {
                             Struct abortedTransactionStruct = partitionDataHeader.instance(ABORTED_TRANSACTIONS_KEY_NAME);
-                            abortedTransactionStruct.set(PRODUCER_ID_KEY_NAME, abortedTransaction.producerId);
-                            abortedTransactionStruct.set(FIRST_OFFSET_KEY_NAME, abortedTransaction.firstOffset);
+                            abortedTransactionStruct.set(PRODUCER_ID, abortedTransaction.producerId);
+                            abortedTransactionStruct.set(FIRST_OFFSET, abortedTransaction.firstOffset);
                             abortedTransactionStructs.add(abortedTransactionStruct);
                         }
                         partitionDataHeader.set(ABORTED_TRANSACTIONS_KEY_NAME, abortedTransactionStructs.toArray());
                     }
                 }
-                if (partitionDataHeader.hasField(LOG_START_OFFSET_KEY_NAME))
-                    partitionDataHeader.set(LOG_START_OFFSET_KEY_NAME, fetchPartitionData.logStartOffset);
-
+                partitionDataHeader.setIfExists(LOG_START_OFFSET, fetchPartitionData.logStartOffset);
                 partitionData.set(PARTITION_HEADER_KEY_NAME, partitionDataHeader);
                 partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.records);
                 partitionArray.add(partitionData);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
index 8fb71a8..2d44ab3 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
@@ -94,13 +94,13 @@ public class FindCoordinatorRequest extends AbstractRequest {
     private final CoordinatorType coordinatorType;
 
     private FindCoordinatorRequest(CoordinatorType coordinatorType, String coordinatorKey, short version) {
-        super(version);
+        super(ApiKeys.FIND_COORDINATOR, version);
         this.coordinatorType = coordinatorType;
         this.coordinatorKey = coordinatorKey;
     }
 
     public FindCoordinatorRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.FIND_COORDINATOR, version);
 
         if (struct.hasField(COORDINATOR_TYPE_KEY_NAME))
             this.coordinatorType = CoordinatorType.forId(struct.getByte(COORDINATOR_TYPE_KEY_NAME));
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
index 5a70ed8..9a13147 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
@@ -80,14 +80,14 @@ public class HeartbeatRequest extends AbstractRequest {
     private final String memberId;
 
     private HeartbeatRequest(String groupId, int groupGenerationId, String memberId, short version) {
-        super(version);
+        super(ApiKeys.HEARTBEAT, version);
         this.groupId = groupId;
         this.groupGenerationId = groupGenerationId;
         this.memberId = memberId;
     }
 
     public HeartbeatRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.HEARTBEAT, version);
         groupId = struct.get(GROUP_ID);
         groupGenerationId = struct.get(GENERATION_ID);
         memberId = struct.get(MEMBER_ID);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
index c350599..aab7c72 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
@@ -82,13 +82,13 @@ public class InitProducerIdRequest extends AbstractRequest {
     }
 
     public InitProducerIdRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.INIT_PRODUCER_ID, version);
         this.transactionalId = struct.get(NULLABLE_TRANSACTIONAL_ID);
         this.transactionTimeoutMs = struct.getInt(TRANSACTION_TIMEOUT_KEY_NAME);
     }
 
     private InitProducerIdRequest(short version, String transactionalId, int transactionTimeoutMs) {
-        super(version);
+        super(ApiKeys.INIT_PRODUCER_ID, version);
         this.transactionalId = transactionalId;
         this.transactionTimeoutMs = transactionTimeoutMs;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
index ba009a1..366509d 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
@@ -160,7 +160,7 @@ public class JoinGroupRequest extends AbstractRequest {
     private JoinGroupRequest(short version, String groupId, int sessionTimeout,
             int rebalanceTimeout, String memberId, String protocolType,
             List<ProtocolMetadata> groupProtocols) {
-        super(version);
+        super(ApiKeys.JOIN_GROUP, version);
         this.groupId = groupId;
         this.sessionTimeout = sessionTimeout;
         this.rebalanceTimeout = rebalanceTimeout;
@@ -170,7 +170,7 @@ public class JoinGroupRequest extends AbstractRequest {
     }
 
     public JoinGroupRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.JOIN_GROUP, versionId);
 
         groupId = struct.get(GROUP_ID);
         sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
index 4fa5337..b821c0e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
@@ -145,7 +145,7 @@ public class LeaderAndIsrRequest extends AbstractRequest {
 
     private LeaderAndIsrRequest(int controllerId, int controllerEpoch, Map<TopicPartition, PartitionState> partitionStates,
                                 Set<Node> liveLeaders, short version) {
-        super(version);
+        super(ApiKeys.LEADER_AND_ISR, version);
         this.controllerId = controllerId;
         this.controllerEpoch = controllerEpoch;
         this.partitionStates = partitionStates;
@@ -153,7 +153,7 @@ public class LeaderAndIsrRequest extends AbstractRequest {
     }
 
     public LeaderAndIsrRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.LEADER_AND_ISR, version);
 
         Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
         for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
index 2b4acf8..2d0b448 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
@@ -74,13 +74,13 @@ public class LeaveGroupRequest extends AbstractRequest {
     private final String memberId;
 
     private LeaveGroupRequest(String groupId, String memberId, short version) {
-        super(version);
+        super(ApiKeys.LEAVE_GROUP, version);
         this.groupId = groupId;
         this.memberId = memberId;
     }
 
     public LeaveGroupRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.LEAVE_GROUP, version);
         groupId = struct.get(GROUP_ID);
         memberId = struct.get(MEMBER_ID);
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
index f64f71a..27254cb 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
@@ -59,11 +59,11 @@ public class ListGroupsRequest extends AbstractRequest {
     }
 
     public ListGroupsRequest(short version) {
-        super(version);
+        super(ApiKeys.LIST_GROUPS, version);
     }
 
     public ListGroupsRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.LIST_GROUPS, versionId);
     }
 
     @Override
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
index be094fe..5107c4e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -32,13 +31,12 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.CURRENT_LEADER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.INT32;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
-import static org.apache.kafka.common.protocol.types.Type.INT8;
 
 public class ListOffsetRequest extends AbstractRequest {
     public static final long EARLIEST_TIMESTAMP = -2L;
@@ -47,70 +45,93 @@ public class ListOffsetRequest extends AbstractRequest {
     public static final int CONSUMER_REPLICA_ID = -1;
     public static final int DEBUGGING_REPLICA_ID = -2;
 
-    private static final String REPLICA_ID_KEY_NAME = "replica_id";
-    private static final String ISOLATION_LEVEL_KEY_NAME = "isolation_level";
-    private static final String TOPICS_KEY_NAME = "topics";
+    // top level fields
+    private static final Field.Int32 REPLICA_ID = new Field.Int32("replica_id",
+            "Broker id of the follower. For normal consumers, use -1.");
+    private static final Field.Int8 ISOLATION_LEVEL = new Field.Int8("isolation_level",
+            "This setting controls the visibility of transactional records. " +
+                    "Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED " +
+                    "(isolation_level = 1), non-transactional and COMMITTED transactional records are visible. " +
+                    "To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current " +
+                    "LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the " +
+                    "result, which allows consumers to discard ABORTED transactional records");
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "Topics to list offsets.");
 
-    // topic level field names
-    private static final String PARTITIONS_KEY_NAME = "partitions";
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "Partitions to list offsets.");
 
-    // partition level field names
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";
-    private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
+    // partition level fields
+    private static final Field.Int64 TIMESTAMP = new Field.Int64("timestamp",
+            "The target timestamp for the partition.");
+    private static final Field.Int32 MAX_NUM_OFFSETS = new Field.Int32("max_num_offsets",
+            "Maximum offsets to return.");
 
-    private static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(
-            PARTITION_ID,
-            new Field(TIMESTAMP_KEY_NAME, INT64, "Timestamp."),
-            new Field(MAX_NUM_OFFSETS_KEY_NAME, INT32, "Maximum offsets to return."));
-    private static final Schema LIST_OFFSET_REQUEST_PARTITION_V1 = new Schema(
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
-            new Field(TIMESTAMP_KEY_NAME, INT64, "The target timestamp for the partition."));
+            TIMESTAMP,
+            MAX_NUM_OFFSETS);
 
-    private static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(
-            TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), "Partitions to list offset."));
-    private static final Schema LIST_OFFSET_REQUEST_TOPIC_V1 = new Schema(
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V1), "Partitions to list offset."));
+            PARTITIONS_V0);
 
     private static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0), "Topics to list offsets."));
+            REPLICA_ID,
+            TOPICS_V0);
+
+    // V1 removes max_num_offsets
+    private static final Field PARTITIONS_V1 = PARTITIONS.withFields(
+            PARTITION_ID,
+            TIMESTAMP);
+
+    private static final Field TOPICS_V1 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V1);
+
     private static final Schema LIST_OFFSET_REQUEST_V1 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1), "Topics to list offsets."));
+            REPLICA_ID,
+            TOPICS_V1);
 
+    // V2 adds a field for the isolation level
     private static final Schema LIST_OFFSET_REQUEST_V2 = new Schema(
-            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
-            new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. " +
-                    "Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED " +
-                    "(isolation_level = 1), non-transactional and COMMITTED transactional records are visible. " +
-                    "To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current " +
-                    "LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the " +
-                    "result, which allows consumers to discard ABORTED transactional records"),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1), "Topics to list offsets."));;
+            REPLICA_ID,
+            ISOLATION_LEVEL,
+            TOPICS_V1);
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V3 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema LIST_OFFSET_REQUEST_V3 = LIST_OFFSET_REQUEST_V2;
 
+    // V4 introduces the current leader epoch, which is used for fencing
+    private static final Field PARTITIONS_V4 = PARTITIONS.withFields(
+            PARTITION_ID,
+            CURRENT_LEADER_EPOCH,
+            TIMESTAMP);
+
+    private static final Field TOPICS_V4 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V4);
+
+    private static final Schema LIST_OFFSET_REQUEST_V4 = new Schema(
+            REPLICA_ID,
+            ISOLATION_LEVEL,
+            TOPICS_V4);
+
     public static Schema[] schemaVersions() {
         return new Schema[] {LIST_OFFSET_REQUEST_V0, LIST_OFFSET_REQUEST_V1, LIST_OFFSET_REQUEST_V2,
-            LIST_OFFSET_REQUEST_V3};
+            LIST_OFFSET_REQUEST_V3, LIST_OFFSET_REQUEST_V4};
     }
 
     private final int replicaId;
     private final IsolationLevel isolationLevel;
-    private final Map<TopicPartition, PartitionData> offsetData;
-    private final Map<TopicPartition, Long> partitionTimestamps;
+    private final Map<TopicPartition, PartitionData> partitionTimestamps;
     private final Set<TopicPartition> duplicatePartitions;
 
     public static class Builder extends AbstractRequest.Builder<ListOffsetRequest> {
         private final int replicaId;
         private final IsolationLevel isolationLevel;
-        private Map<TopicPartition, PartitionData> offsetData = null;
-        private Map<TopicPartition, Long> partitionTimestamps = null;
+        private Map<TopicPartition, PartitionData> partitionTimestamps = new HashMap<>();
 
         public static Builder forReplica(short allowedVersion, int replicaId) {
             return new Builder((short) 0, allowedVersion, replicaId, IsolationLevel.READ_UNCOMMITTED);
@@ -125,49 +146,23 @@ public class ListOffsetRequest extends AbstractRequest {
             return new Builder(minVersion, ApiKeys.LIST_OFFSETS.latestVersion(), CONSUMER_REPLICA_ID, isolationLevel);
         }
 
-        private Builder(short oldestAllowedVersion, short latestAllowedVersion, int replicaId, IsolationLevel isolationLevel) {
+        private Builder(short oldestAllowedVersion,
+                        short latestAllowedVersion,
+                        int replicaId,
+                        IsolationLevel isolationLevel) {
             super(ApiKeys.LIST_OFFSETS, oldestAllowedVersion, latestAllowedVersion);
             this.replicaId = replicaId;
             this.isolationLevel = isolationLevel;
         }
 
-        public Builder setOffsetData(Map<TopicPartition, PartitionData> offsetData) {
-            this.offsetData = offsetData;
-            return this;
-        }
-
-        public Builder setTargetTimes(Map<TopicPartition, Long> partitionTimestamps) {
+        public Builder setTargetTimes(Map<TopicPartition, PartitionData> partitionTimestamps) {
             this.partitionTimestamps = partitionTimestamps;
             return this;
         }
 
         @Override
         public ListOffsetRequest build(short version) {
-            if (version == 0) {
-                if (offsetData == null) {
-                    if (partitionTimestamps == null) {
-                        throw new RuntimeException("Must set partitionTimestamps or offsetData when creating a v0 " +
-                            "ListOffsetRequest");
-                    } else {
-                        offsetData = new HashMap<>();
-                        for (Map.Entry<TopicPartition, Long> entry: partitionTimestamps.entrySet()) {
-                            offsetData.put(entry.getKey(),
-                                    new PartitionData(entry.getValue(), 1));
-                        }
-                        this.partitionTimestamps = null;
-                    }
-                }
-            } else {
-                if (offsetData != null) {
-                    throw new RuntimeException("Cannot create a v" + version + " ListOffsetRequest with v0 " +
-                        "PartitionData.");
-                } else if (partitionTimestamps == null) {
-                    throw new RuntimeException("Must set partitionTimestamps when creating a v" +
-                            version + " ListOffsetRequest");
-                }
-            }
-            Map<TopicPartition, ?> m = (version == 0) ?  offsetData : partitionTimestamps;
-            return new ListOffsetRequest(replicaId, m, isolationLevel, version);
+            return new ListOffsetRequest(replicaId, partitionTimestamps, isolationLevel, version);
         }
 
         @Override
@@ -175,9 +170,6 @@ public class ListOffsetRequest extends AbstractRequest {
             StringBuilder bld = new StringBuilder();
             bld.append("(type=ListOffsetRequest")
                .append(", replicaId=").append(replicaId);
-            if (offsetData != null) {
-                bld.append(", offsetData=").append(offsetData);
-            }
             if (partitionTimestamps != null) {
                 bld.append(", partitionTimestamps=").append(partitionTimestamps);
             }
@@ -187,25 +179,34 @@ public class ListOffsetRequest extends AbstractRequest {
         }
     }
 
-    /**
-     * This class is only used by ListOffsetRequest v0 which has been deprecated.
-     */
-    @Deprecated
     public static final class PartitionData {
         public final long timestamp;
-        public final int maxNumOffsets;
+        @Deprecated
+        public final int maxNumOffsets; // only supported in v0
+        public final Optional<Integer> currentLeaderEpoch;
 
-        public PartitionData(long timestamp, int maxNumOffsets) {
+        private PartitionData(long timestamp, int maxNumOffsets, Optional<Integer> currentLeaderEpoch) {
             this.timestamp = timestamp;
             this.maxNumOffsets = maxNumOffsets;
+            this.currentLeaderEpoch = currentLeaderEpoch;
+        }
+
+        @Deprecated
+        public PartitionData(long timestamp, int maxNumOffsets) {
+            this(timestamp, maxNumOffsets, Optional.empty());
+        }
+
+        public PartitionData(long timestamp, Optional<Integer> currentLeaderEpoch) {
+            this(timestamp, 1, currentLeaderEpoch);
         }
 
         @Override
         public String toString() {
             StringBuilder bld = new StringBuilder();
             bld.append("{timestamp: ").append(timestamp).
-                append(", maxNumOffsets: ").append(maxNumOffsets).
-                append("}");
+                    append(", maxNumOffsets: ").append(maxNumOffsets).
+                    append(", currentLeaderEpoch: ").append(currentLeaderEpoch).
+                    append("}");
             return bld.toString();
         }
     }
@@ -214,40 +215,39 @@ public class ListOffsetRequest extends AbstractRequest {
      * Private constructor with a specified version.
      */
     @SuppressWarnings("unchecked")
-    private ListOffsetRequest(int replicaId, Map<TopicPartition, ?> targetTimes, IsolationLevel isolationLevel, short version) {
-        super(version);
+    private ListOffsetRequest(int replicaId,
+                              Map<TopicPartition, PartitionData> targetTimes,
+                              IsolationLevel isolationLevel,
+                              short version) {
+        super(ApiKeys.LIST_OFFSETS, version);
         this.replicaId = replicaId;
         this.isolationLevel = isolationLevel;
-        this.offsetData = version == 0 ? (Map<TopicPartition, PartitionData>) targetTimes : null;
-        this.partitionTimestamps = version >= 1 ? (Map<TopicPartition, Long>) targetTimes : null;
+        this.partitionTimestamps = targetTimes;
         this.duplicatePartitions = Collections.emptySet();
     }
 
     public ListOffsetRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.LIST_OFFSETS, version);
         Set<TopicPartition> duplicatePartitions = new HashSet<>();
-        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
-        isolationLevel = struct.hasField(ISOLATION_LEVEL_KEY_NAME) ?
-                IsolationLevel.forId(struct.getByte(ISOLATION_LEVEL_KEY_NAME)) :
+        replicaId = struct.get(REPLICA_ID);
+        isolationLevel = struct.hasField(ISOLATION_LEVEL) ?
+                IsolationLevel.forId(struct.get(ISOLATION_LEVEL)) :
                 IsolationLevel.READ_UNCOMMITTED;
-        offsetData = new HashMap<>();
         partitionTimestamps = new HashMap<>();
-        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+        for (Object topicResponseObj : struct.get(TOPICS)) {
             Struct topicResponse = (Struct) topicResponseObj;
             String topic = topicResponse.get(TOPIC_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionResponseObj : topicResponse.get(PARTITIONS)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
                 int partition = partitionResponse.get(PARTITION_ID);
-                long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
+                long timestamp = partitionResponse.get(TIMESTAMP);
                 TopicPartition tp = new TopicPartition(topic, partition);
-                if (partitionResponse.hasField(MAX_NUM_OFFSETS_KEY_NAME)) {
-                    int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME);
-                    PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets);
-                    offsetData.put(tp, partitionData);
-                } else {
-                    if (partitionTimestamps.put(tp, timestamp) != null)
-                        duplicatePartitions.add(tp);
-                }
+
+                int maxNumOffsets = partitionResponse.getOrElse(MAX_NUM_OFFSETS, 1);
+                Optional<Integer> currentLeaderEpoch = RequestUtils.getLeaderEpoch(partitionResponse, CURRENT_LEADER_EPOCH);
+                PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets, currentLeaderEpoch);
+                if (partitionTimestamps.put(tp, partitionData) != null)
+                    duplicatePartitions.add(tp);
             }
         }
         this.duplicatePartitions = duplicatePartitions;
@@ -257,27 +257,21 @@ public class ListOffsetRequest extends AbstractRequest {
     @SuppressWarnings("deprecation")
     public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
         Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<>();
-
         short versionId = version();
-        if (versionId == 0) {
-            for (Map.Entry<TopicPartition, PartitionData> entry : offsetData.entrySet()) {
-                ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(
-                        Errors.forException(e), Collections.<Long>emptyList());
-                responseData.put(entry.getKey(), partitionResponse);
-            }
-        } else {
-            for (Map.Entry<TopicPartition, Long> entry : partitionTimestamps.entrySet()) {
-                ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(
-                        Errors.forException(e), -1L, -1L);
-                responseData.put(entry.getKey(), partitionResponse);
-            }
+
+        ListOffsetResponse.PartitionData partitionError = versionId == 0 ?
+                new ListOffsetResponse.PartitionData(Errors.forException(e), Collections.emptyList()) :
+                new ListOffsetResponse.PartitionData(Errors.forException(e), -1L, -1L, Optional.empty());
+        for (TopicPartition partition : partitionTimestamps.keySet()) {
+            responseData.put(partition, partitionError);
         }
 
-        switch (versionId) {
+        switch (version()) {
             case 0:
             case 1:
             case 2:
             case 3:
+            case 4:
                 return new ListOffsetResponse(throttleTimeMs, responseData);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
@@ -293,12 +287,7 @@ public class ListOffsetRequest extends AbstractRequest {
         return isolationLevel;
     }
 
-    @Deprecated
-    public Map<TopicPartition, PartitionData> offsetData() {
-        return offsetData;
-    }
-
-    public Map<TopicPartition, Long> partitionTimestamps() {
+    public Map<TopicPartition, PartitionData> partitionTimestamps() {
         return partitionTimestamps;
     }
 
@@ -314,39 +303,30 @@ public class ListOffsetRequest extends AbstractRequest {
     protected Struct toStruct() {
         short version = version();
         Struct struct = new Struct(ApiKeys.LIST_OFFSETS.requestSchema(version));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupPartitionDataByTopic(partitionTimestamps);
 
-        Map<TopicPartition, ?> targetTimes = partitionTimestamps == null ? offsetData : partitionTimestamps;
-        Map<String, Map<Integer, Object>> topicsData = CollectionUtils.groupDataByTopic(targetTimes);
-
-        struct.set(REPLICA_ID_KEY_NAME, replicaId);
+        struct.set(REPLICA_ID, replicaId);
+        struct.setIfExists(ISOLATION_LEVEL, isolationLevel.id());
 
-        if (struct.hasField(ISOLATION_LEVEL_KEY_NAME))
-            struct.set(ISOLATION_LEVEL_KEY_NAME, isolationLevel.id());
         List<Struct> topicArray = new ArrayList<>();
-        for (Map.Entry<String, Map<Integer, Object>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS);
             topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
-            for (Map.Entry<Integer, Object> partitionEntry : topicEntry.getValue().entrySet()) {
-                if (version == 0) {
-                    PartitionData offsetPartitionData = (PartitionData) partitionEntry.getValue();
-                    Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                    partitionData.set(PARTITION_ID, partitionEntry.getKey());
-                    partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
-                    partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
-                    partitionArray.add(partitionData);
-                } else {
-                    Long timestamp = (Long) partitionEntry.getValue();
-                    Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                    partitionData.set(PARTITION_ID, partitionEntry.getKey());
-                    partitionData.set(TIMESTAMP_KEY_NAME, timestamp);
-                    partitionArray.add(partitionData);
-                }
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData offsetPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS);
+                partitionData.set(PARTITION_ID, partitionEntry.getKey());
+                partitionData.set(TIMESTAMP, offsetPartitionData.timestamp);
+                partitionData.setIfExists(MAX_NUM_OFFSETS, offsetPartitionData.maxNumOffsets);
+                RequestUtils.setLeaderEpochIfExists(partitionData, CURRENT_LEADER_EPOCH,
+                        offsetPartitionData.currentLeaderEpoch);
+                partitionArray.add(partitionData);
             }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicData.set(PARTITIONS, partitionArray.toArray());
             topicArray.add(topicData);
         }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        struct.set(TOPICS, topicArray.toArray());
         return struct;
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
index e719dbb..9f3ce73 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -31,73 +30,98 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
 import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.LEADER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 import static org.apache.kafka.common.protocol.types.Type.INT64;
 
+/**
+ * Possible error code:
+ *
+ *  UNKNOWN_TOPIC_OR_PARTITION (3)
+ *  NOT_LEADER_FOR_PARTITION (6)
+ *  UNSUPPORTED_FOR_MESSAGE_FORMAT (43)
+ *  UNKNOWN (-1)
+ */
 public class ListOffsetResponse extends AbstractResponse {
     public static final long UNKNOWN_TIMESTAMP = -1L;
     public static final long UNKNOWN_OFFSET = -1L;
 
-    private static final String RESPONSES_KEY_NAME = "responses";
+    // top level fields
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("responses",
+            "The listed offsets by topic");
 
-    // topic level field names
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    /**
-     * Possible error code:
-     *
-     *  UNKNOWN_TOPIC_OR_PARTITION (3)
-     *  NOT_LEADER_FOR_PARTITION (6)
-     *  UNSUPPORTED_FOR_MESSAGE_FORMAT (43)
-     *  UNKNOWN (-1)
-     */
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partition_responses",
+            "The listed offsets by partition");
 
+    // partition level fields
     // This key is only used by ListOffsetResponse v0
     @Deprecated
-    private static final String OFFSETS_KEY_NAME = "offsets";
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";
-    private static final String OFFSET_KEY_NAME = "offset";
+    private static final Field.Array OFFSETS = new Field.Array("offsets'", INT64, "A list of offsets.");
+    private static final Field.Int64 TIMESTAMP = new Field.Int64("timestamp",
+            "The timestamp associated with the returned offset");
+    private static final Field.Int64 OFFSET = new Field.Int64("offset",
+            "The offset found");
 
-    private static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
             ERROR_CODE,
-            new Field(OFFSETS_KEY_NAME, new ArrayOf(INT64), "A list of offsets."));
+            OFFSETS);
+
+    private static final Field TOPICS_V0 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V0);
 
-    private static final Schema LIST_OFFSET_RESPONSE_PARTITION_V1 = new Schema(
+    private static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(
+            TOPICS_V0);
+
+    // V1 bumped for the removal of the offsets array
+    private static final Field PARTITIONS_V1 = PARTITIONS.withFields(
             PARTITION_ID,
             ERROR_CODE,
-            new Field(TIMESTAMP_KEY_NAME, INT64, "The timestamp associated with the returned offset"),
-            new Field(OFFSET_KEY_NAME, INT64, "offset found"));
-
-    private static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(
-            TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
+            TIMESTAMP,
+            OFFSET);
 
-    private static final Schema LIST_OFFSET_RESPONSE_TOPIC_V1 = new Schema(
+    private static final Field TOPICS_V1 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V1)));
-
-    private static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
+            PARTITIONS_V1);
 
     private static final Schema LIST_OFFSET_RESPONSE_V1 = new Schema(
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V1)));
+            TOPICS_V1);
+
+    // V2 bumped for the addition of the throttle time
     private static final Schema LIST_OFFSET_RESPONSE_V2 = new Schema(
             THROTTLE_TIME_MS,
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V1)));
+            TOPICS_V1);
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V3 bumped to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema LIST_OFFSET_RESPONSE_V3 = LIST_OFFSET_RESPONSE_V2;
 
+    // V4 bumped for the addition of the current leader epoch in the request schema and the
+    // leader epoch in the response partition data
+    private static final Field PARTITIONS_V4 = PARTITIONS.withFields(
+            PARTITION_ID,
+            ERROR_CODE,
+            TIMESTAMP,
+            OFFSET,
+            LEADER_EPOCH);
+
+    private static final Field TOPICS_V4 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V4);
+
+    private static final Schema LIST_OFFSET_RESPONSE_V4 = new Schema(
+            THROTTLE_TIME_MS,
+            TOPICS_V4);
+
     public static Schema[] schemaVersions() {
         return new Schema[] {LIST_OFFSET_RESPONSE_V0, LIST_OFFSET_RESPONSE_V1, LIST_OFFSET_RESPONSE_V2,
-            LIST_OFFSET_RESPONSE_V3};
+            LIST_OFFSET_RESPONSE_V3, LIST_OFFSET_RESPONSE_V4};
     }
 
     public static final class PartitionData {
@@ -107,6 +131,7 @@ public class ListOffsetResponse extends AbstractResponse {
         public final List<Long> offsets;
         public final Long timestamp;
         public final Long offset;
+        public final Optional<Integer> leaderEpoch;
 
         /**
          * Constructor for ListOffsetResponse v0
@@ -117,32 +142,37 @@ public class ListOffsetResponse extends AbstractResponse {
             this.offsets = offsets;
             this.timestamp = null;
             this.offset = null;
+            this.leaderEpoch = Optional.empty();
         }
 
         /**
          * Constructor for ListOffsetResponse v1
          */
-        public PartitionData(Errors error, long timestamp, long offset) {
+        public PartitionData(Errors error, long timestamp, long offset, Optional<Integer> leaderEpoch) {
             this.error = error;
             this.timestamp = timestamp;
             this.offset = offset;
             this.offsets = null;
+            this.leaderEpoch = leaderEpoch;
         }
 
         @Override
         public String toString() {
             StringBuilder bld = new StringBuilder();
-            bld.append("PartitionData{").
-                append("errorCode: ").append((int) error.code()).
-                append(", timestamp: ").append(timestamp).
-                append(", offset: ").append(offset).
-                append(", offsets: ");
+            bld.append("PartitionData(").
+                    append("errorCode: ").append((int) error.code());
+
             if (offsets == null) {
-                bld.append(offsets);
+                bld.append(", timestamp: ").append(timestamp).
+                        append(", offset: ").append(offset).
+                        append(", leaderEpoch: ").append(leaderEpoch);
             } else {
-                bld.append("[").append(Utils.join(this.offsets, ",")).append("]");
+                bld.append(", offsets: ").
+                        append("[").
+                        append(Utils.join(this.offsets, ",")).
+                        append("]");
             }
-            bld.append("}");
+            bld.append(")");
             return bld.toString();
         }
     }
@@ -165,24 +195,25 @@ public class ListOffsetResponse extends AbstractResponse {
     public ListOffsetResponse(Struct struct) {
         this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         responseData = new HashMap<>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+        for (Object topicResponseObj : struct.get(TOPICS)) {
             Struct topicResponse = (Struct) topicResponseObj;
             String topic = topicResponse.get(TOPIC_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionResponseObj : topicResponse.get(PARTITIONS)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
                 int partition = partitionResponse.get(PARTITION_ID);
                 Errors error = Errors.forCode(partitionResponse.get(ERROR_CODE));
                 PartitionData partitionData;
-                if (partitionResponse.hasField(OFFSETS_KEY_NAME)) {
-                    Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
+                if (partitionResponse.hasField(OFFSETS)) {
+                    Object[] offsets = partitionResponse.get(OFFSETS);
                     List<Long> offsetsList = new ArrayList<>();
                     for (Object offset : offsets)
                         offsetsList.add((Long) offset);
                     partitionData = new PartitionData(error, offsetsList);
                 } else {
-                    long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
-                    long offset = partitionResponse.getLong(OFFSET_KEY_NAME);
-                    partitionData = new PartitionData(error, timestamp, offset);
+                    long timestamp = partitionResponse.get(TIMESTAMP);
+                    long offset = partitionResponse.get(OFFSET);
+                    Optional<Integer> leaderEpoch = RequestUtils.getLeaderEpoch(partitionResponse, LEADER_EPOCH);
+                    partitionData = new PartitionData(error, timestamp, offset, leaderEpoch);
                 }
                 responseData.put(new TopicPartition(topic, partition), partitionData);
             }
@@ -214,30 +245,31 @@ public class ListOffsetResponse extends AbstractResponse {
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.LIST_OFFSETS.responseSchema(version));
         struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupPartitionDataByTopic(responseData);
 
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            Struct topicData = struct.instance(TOPICS);
             topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
                 PartitionData offsetPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                Struct partitionData = topicData.instance(PARTITIONS);
                 partitionData.set(PARTITION_ID, partitionEntry.getKey());
                 partitionData.set(ERROR_CODE, offsetPartitionData.error.code());
-                if (version == 0)
-                    partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
-                else {
-                    partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
-                    partitionData.set(OFFSET_KEY_NAME, offsetPartitionData.offset);
+                if (version == 0) {
+                    partitionData.set(OFFSETS, offsetPartitionData.offsets.toArray());
+                } else {
+                    partitionData.set(TIMESTAMP, offsetPartitionData.timestamp);
+                    partitionData.set(OFFSET, offsetPartitionData.offset);
+                    RequestUtils.setLeaderEpochIfExists(partitionData, LEADER_EPOCH, offsetPartitionData.leaderEpoch);
                 }
                 partitionArray.add(partitionData);
             }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicData.set(PARTITIONS, partitionArray.toArray());
             topicArray.add(topicData);
         }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        struct.set(TOPICS, topicArray.toArray());
 
         return struct;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
index 67dbe94..89a6e69 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.common.requests;
 
-import org.apache.kafka.common.Node;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
@@ -31,13 +30,11 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
 import static org.apache.kafka.common.protocol.types.Type.STRING;
 
 public class MetadataRequest extends AbstractRequest {
 
     private static final String TOPICS_KEY_NAME = "topics";
-    private static final String ALLOW_AUTO_TOPIC_CREATION_KEY_NAME = "allow_auto_topic_creation";
 
     private static final Schema METADATA_REQUEST_V0 = new Schema(
             new Field(TOPICS_KEY_NAME, new ArrayOf(STRING), "An array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics."));
@@ -52,12 +49,14 @@ public class MetadataRequest extends AbstractRequest {
     private static final Schema METADATA_REQUEST_V3 = METADATA_REQUEST_V2;
 
     /* The v4 metadata request has an additional field for allowing auto topic creation. The response is the same as v3. */
+    private static final Field.Bool ALLOW_AUTO_TOPIC_CREATION = new Field.Bool("allow_auto_topic_creation",
+            "If this and the broker config <code>auto.create.topics.enable</code> are true, topics that " +
+                    "don't exist will be created by the broker. Otherwise, no topics will be created by the broker.");
+
     private static final Schema METADATA_REQUEST_V4 = new Schema(
             new Field(TOPICS_KEY_NAME, ArrayOf.nullable(STRING), "An array of topics to fetch metadata for. " +
                     "If the topics array is null fetch metadata for all topics."),
-            new Field(ALLOW_AUTO_TOPIC_CREATION_KEY_NAME, BOOLEAN, "If this and the broker config " +
-                    "'auto.create.topics.enable' are true, topics that don't exist will be created by the broker. " +
-                    "Otherwise, no topics will be created by the broker."));
+            ALLOW_AUTO_TOPIC_CREATION);
 
     /* The v5 metadata request is the same as v4. An additional field for offline_replicas has been added to the v5 metadata response */
     private static final Schema METADATA_REQUEST_V5 = METADATA_REQUEST_V4;
@@ -67,9 +66,14 @@ public class MetadataRequest extends AbstractRequest {
      */
     private static final Schema METADATA_REQUEST_V6 = METADATA_REQUEST_V5;
 
+    /**
+     * Bumped for the addition of the current leader epoch in the metadata response.
+     */
+    private static final Schema METADATA_REQUEST_V7 = METADATA_REQUEST_V6;
+
     public static Schema[] schemaVersions() {
         return new Schema[] {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3,
-            METADATA_REQUEST_V4, METADATA_REQUEST_V5, METADATA_REQUEST_V6};
+            METADATA_REQUEST_V4, METADATA_REQUEST_V5, METADATA_REQUEST_V6, METADATA_REQUEST_V7};
     }
 
     public static class Builder extends AbstractRequest.Builder<MetadataRequest> {
@@ -133,13 +137,13 @@ public class MetadataRequest extends AbstractRequest {
      * In v1 null indicates requesting all topics, and an empty list indicates requesting no topics.
      */
     public MetadataRequest(List<String> topics, boolean allowAutoTopicCreation, short version) {
-        super(version);
+        super(ApiKeys.METADATA, version);
         this.topics = topics;
         this.allowAutoTopicCreation = allowAutoTopicCreation;
     }
 
     public MetadataRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.METADATA, version);
         Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
         if (topicArray != null) {
             topics = new ArrayList<>();
@@ -149,10 +153,8 @@ public class MetadataRequest extends AbstractRequest {
         } else {
             topics = null;
         }
-        if (struct.hasField(ALLOW_AUTO_TOPIC_CREATION_KEY_NAME))
-            allowAutoTopicCreation = struct.getBoolean(ALLOW_AUTO_TOPIC_CREATION_KEY_NAME);
-        else
-            allowAutoTopicCreation = true;
+
+        allowAutoTopicCreation = struct.getOrElse(ALLOW_AUTO_TOPIC_CREATION, true);
     }
 
     @Override
@@ -171,12 +173,13 @@ public class MetadataRequest extends AbstractRequest {
             case 0:
             case 1:
             case 2:
-                return new MetadataResponse(Collections.<Node>emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas);
+                return new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas);
             case 3:
             case 4:
             case 5:
             case 6:
-                return new MetadataResponse(throttleTimeMs, Collections.<Node>emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas);
+            case 7:
+                return new MetadataResponse(throttleTimeMs, Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
                         versionId, this.getClass().getSimpleName(), ApiKeys.METADATA.latestVersion()));
@@ -206,8 +209,7 @@ public class MetadataRequest extends AbstractRequest {
             struct.set(TOPICS_KEY_NAME, null);
         else
             struct.set(TOPICS_KEY_NAME, topics.toArray());
-        if (struct.hasField(ALLOW_AUTO_TOPIC_CREATION_KEY_NAME))
-            struct.set(ALLOW_AUTO_TOPIC_CREATION_KEY_NAME, allowAutoTopicCreation);
+        struct.setIfExists(ALLOW_AUTO_TOPIC_CREATION, allowAutoTopicCreation);
         return struct;
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
index 09a04e5..c78066f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
@@ -22,7 +22,6 @@ import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.errors.InvalidMetadataException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -35,143 +34,170 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
 import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.LEADER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
 import static org.apache.kafka.common.protocol.types.Type.INT32;
-import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
-import static org.apache.kafka.common.protocol.types.Type.STRING;
 
-public class MetadataResponse extends AbstractResponse {
-    private static final String BROKERS_KEY_NAME = "brokers";
-    private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata";
-
-    // broker level field names
-    private static final String NODE_ID_KEY_NAME = "node_id";
-    private static final String HOST_KEY_NAME = "host";
-    private static final String PORT_KEY_NAME = "port";
-    private static final String RACK_KEY_NAME = "rack";
+/**
+ * Possible topic-level error codes:
+ *  UnknownTopic (3)
+ *  LeaderNotAvailable (5)
+ *  InvalidTopic (17)
+ *  TopicAuthorizationFailed (29)
 
-    private static final String CONTROLLER_ID_KEY_NAME = "controller_id";
+ * Possible partition-level error codes:
+ *  LeaderNotAvailable (5)
+ *  ReplicaNotAvailable (9)
+ */
+public class MetadataResponse extends AbstractResponse {
     public static final int NO_CONTROLLER_ID = -1;
 
-    private static final String CLUSTER_ID_KEY_NAME = "cluster_id";
-
-    /**
-     * Possible error codes:
-     *
-     * UnknownTopic (3)
-     * LeaderNotAvailable (5)
-     * InvalidTopic (17)
-     * TopicAuthorizationFailed (29)
-     */
-
-    private static final String IS_INTERNAL_KEY_NAME = "is_internal";
-    private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata";
-
-    /**
-     * Possible error codes:
-     *
-     * LeaderNotAvailable (5)
-     * ReplicaNotAvailable (9)
-     */
-
-    private static final String LEADER_KEY_NAME = "leader";
-    private static final String REPLICAS_KEY_NAME = "replicas";
-    private static final String ISR_KEY_NAME = "isr";
-    private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas";
-
-    private static final Schema METADATA_BROKER_V0 = new Schema(
-            new Field(NODE_ID_KEY_NAME, INT32, "The broker id."),
-            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
-            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."));
-
-    private static final Schema PARTITION_METADATA_V0 = new Schema(
+    private static final Field.ComplexArray BROKERS = new Field.ComplexArray("brokers",
+            "Host and port information for all brokers.");
+    private static final Field.ComplexArray TOPIC_METADATA = new Field.ComplexArray("topic_metadata",
+            "Metadata for requested topics");
+
+    // cluster level fields
+    private static final Field.NullableStr CLUSTER_ID = new Field.NullableStr("cluster_id",
+            "The cluster id that this broker belongs to.");
+    private static final Field.Int32 CONTROLLER_ID = new Field.Int32("controller_id",
+            "The broker id of the controller broker.");
+
+    // broker level fields
+    private static final Field.Int32 NODE_ID = new Field.Int32("node_id", "The broker id.");
+    private static final Field.Str HOST = new Field.Str("host", "The hostname of the broker.");
+    private static final Field.Int32 PORT = new Field.Int32("port", "The port on which the broker accepts requests.");
+    private static final Field.NullableStr RACK = new Field.NullableStr("rack", "The rack of the broker.");
+
+    // topic level fields
+    private static final Field.ComplexArray PARTITION_METADATA = new Field.ComplexArray("partition_metadata",
+            "Metadata for each partition of the topic.");
+    private static final Field.Bool IS_INTERNAL = new Field.Bool("is_internal",
+            "Indicates if the topic is considered a Kafka internal topic");
+
+    // partition level fields
+    private static final Field.Int32 LEADER = new Field.Int32("leader",
+            "The id of the broker acting as leader for this partition.");
+    private static final Field.Array REPLICAS = new Field.Array("replicas", INT32,
+            "The set of all nodes that host this partition.");
+    private static final Field.Array ISR = new Field.Array("isr", INT32,
+            "The set of nodes that are in sync with the leader for this partition.");
+    private static final Field.Array OFFLINE_REPLICAS = new Field.Array("offline_replicas", INT32,
+            "The set of offline replicas of this partition.");
+
+    private static final Field METADATA_BROKER_V0 = BROKERS.withFields(
+            NODE_ID,
+            HOST,
+            PORT);
+
+    private static final Field PARTITION_METADATA_V0 = PARTITION_METADATA.withFields(
             ERROR_CODE,
             PARTITION_ID,
-            new Field(LEADER_KEY_NAME, INT32, "The id of the broker acting as leader for this partition."),
-            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of all nodes that host this partition."),
-            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The set of nodes that are in sync with the leader for this partition."));
+            LEADER,
+            REPLICAS,
+            ISR);
 
-    private static final Schema TOPIC_METADATA_V0 = new Schema(
+    private static final Field TOPIC_METADATA_V0 = TOPIC_METADATA.withFields(
             ERROR_CODE,
             TOPIC_NAME,
-            new Field(PARTITION_METADATA_KEY_NAME, new ArrayOf(PARTITION_METADATA_V0), "Metadata for each partition of the topic."));
+            PARTITION_METADATA_V0);
 
     private static final Schema METADATA_RESPONSE_V0 = new Schema(
-            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V0), "Host and port information for all brokers."),
-            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V0)));
+            METADATA_BROKER_V0,
+            TOPIC_METADATA_V0);
 
-    private static final Schema METADATA_BROKER_V1 = new Schema(
-            new Field(NODE_ID_KEY_NAME, INT32, "The broker id."),
-            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
-            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."),
-            new Field(RACK_KEY_NAME, NULLABLE_STRING, "The rack of the broker."));
+    // V1 adds fields for the rack of each broker, the controller id, and whether or not the topic is internal
+    private static final Field METADATA_BROKER_V1 = BROKERS.withFields(
+            NODE_ID,
+            HOST,
+            PORT,
+            RACK);
 
-    private static final Schema PARTITION_METADATA_V1 = PARTITION_METADATA_V0;
-
-    // PARTITION_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
-    private static final Schema PARTITION_METADATA_V2 = new Schema(
-            ERROR_CODE,
-            PARTITION_ID,
-            new Field(LEADER_KEY_NAME, INT32, "The id of the broker acting as leader for this partition."),
-            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of all nodes that host this partition."),
-            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The set of nodes that are in sync with the leader for this partition."),
-            new Field(OFFLINE_REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of offline replicas of this partition."));
-
-    private static final Schema TOPIC_METADATA_V1 = new Schema(
+    private static final Field TOPIC_METADATA_V1 = TOPIC_METADATA.withFields(
             ERROR_CODE,
             TOPIC_NAME,
-            new Field(IS_INTERNAL_KEY_NAME, BOOLEAN, "Indicates if the topic is considered a Kafka internal topic"),
-            new Field(PARTITION_METADATA_KEY_NAME, new ArrayOf(PARTITION_METADATA_V1), "Metadata for each partition of the topic."));
-
-    // TOPIC_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
-    private static final Schema TOPIC_METADATA_V2 = new Schema(
-            ERROR_CODE,
-            TOPIC_NAME,
-            new Field(IS_INTERNAL_KEY_NAME, BOOLEAN, "Indicates if the topic is considered a Kafka internal topic"),
-            new Field(PARTITION_METADATA_KEY_NAME, new ArrayOf(PARTITION_METADATA_V2), "Metadata for each partition of the topic."));
+            IS_INTERNAL,
+            PARTITION_METADATA_V0);
 
     private static final Schema METADATA_RESPONSE_V1 = new Schema(
-            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
-            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
-            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V1)));
+            METADATA_BROKER_V1,
+            CONTROLLER_ID,
+            TOPIC_METADATA_V1);
 
+    // V2 added a field for the cluster id
     private static final Schema METADATA_RESPONSE_V2 = new Schema(
-            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
-            new Field(CLUSTER_ID_KEY_NAME, NULLABLE_STRING, "The cluster id that this broker belongs to."),
-            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
-            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V1)));
+            METADATA_BROKER_V1,
+            CLUSTER_ID,
+            CONTROLLER_ID,
+            TOPIC_METADATA_V1);
 
+    // V3 adds the throttle time to the response
     private static final Schema METADATA_RESPONSE_V3 = new Schema(
             THROTTLE_TIME_MS,
-            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
-            new Field(CLUSTER_ID_KEY_NAME, NULLABLE_STRING, "The cluster id that this broker belongs to."),
-            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
-            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V1)));
+            METADATA_BROKER_V1,
+            CLUSTER_ID,
+            CONTROLLER_ID,
+            TOPIC_METADATA_V1);
 
     private static final Schema METADATA_RESPONSE_V4 = METADATA_RESPONSE_V3;
 
-    // METADATA_RESPONSE_V5 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    // V5 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    private static final Field PARTITION_METADATA_V5 = PARTITION_METADATA.withFields(
+            ERROR_CODE,
+            PARTITION_ID,
+            LEADER,
+            REPLICAS,
+            ISR,
+            OFFLINE_REPLICAS);
+
+    private static final Field TOPIC_METADATA_V5 = TOPIC_METADATA.withFields(
+            ERROR_CODE,
+            TOPIC_NAME,
+            IS_INTERNAL,
+            PARTITION_METADATA_V5);
+
     private static final Schema METADATA_RESPONSE_V5 = new Schema(
             THROTTLE_TIME_MS,
-            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
-            new Field(CLUSTER_ID_KEY_NAME, NULLABLE_STRING, "The cluster id that this broker belongs to."),
-            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
-            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V2)));
+            METADATA_BROKER_V1,
+            CLUSTER_ID,
+            CONTROLLER_ID,
+            TOPIC_METADATA_V5);
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V6 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema METADATA_RESPONSE_V6 = METADATA_RESPONSE_V5;
 
+    // V7 adds the leader epoch to the partition metadata
+    private static final Field PARTITION_METADATA_V7 = PARTITION_METADATA.withFields(
+            ERROR_CODE,
+            PARTITION_ID,
+            LEADER,
+            LEADER_EPOCH,
+            REPLICAS,
+            ISR,
+            OFFLINE_REPLICAS);
+
+    private static final Field TOPIC_METADATA_V7 = TOPIC_METADATA.withFields(
+            ERROR_CODE,
+            TOPIC_NAME,
+            IS_INTERNAL,
+            PARTITION_METADATA_V7);
+
+    private static final Schema METADATA_RESPONSE_V7 = new Schema(
+            THROTTLE_TIME_MS,
+            METADATA_BROKER_V1,
+            CLUSTER_ID,
+            CONTROLLER_ID,
+            TOPIC_METADATA_V7);
+
     public static Schema[] schemaVersions() {
         return new Schema[] {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3,
-            METADATA_RESPONSE_V4, METADATA_RESPONSE_V5, METADATA_RESPONSE_V6};
+            METADATA_RESPONSE_V4, METADATA_RESPONSE_V5, METADATA_RESPONSE_V6, METADATA_RESPONSE_V7};
     }
 
     private final int throttleTimeMs;
@@ -198,62 +224,56 @@ public class MetadataResponse extends AbstractResponse {
     public MetadataResponse(Struct struct) {
         this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         Map<Integer, Node> brokers = new HashMap<>();
-        Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
+        Object[] brokerStructs = struct.get(BROKERS);
         for (Object brokerStruct : brokerStructs) {
             Struct broker = (Struct) brokerStruct;
-            int nodeId = broker.getInt(NODE_ID_KEY_NAME);
-            String host = broker.getString(HOST_KEY_NAME);
-            int port = broker.getInt(PORT_KEY_NAME);
+            int nodeId = broker.get(NODE_ID);
+            String host = broker.get(HOST);
+            int port = broker.get(PORT);
             // This field only exists in v1+
             // When we can't know if a rack exists in a v0 response we default to null
-            String rack = broker.hasField(RACK_KEY_NAME) ? broker.getString(RACK_KEY_NAME) : null;
+            String rack = broker.getOrElse(RACK, null);
             brokers.put(nodeId, new Node(nodeId, host, port, rack));
         }
 
         // This field only exists in v1+
         // When we can't know the controller id in a v0 response we default to NO_CONTROLLER_ID
-        int controllerId = NO_CONTROLLER_ID;
-        if (struct.hasField(CONTROLLER_ID_KEY_NAME))
-            controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME);
+        int controllerId = struct.getOrElse(CONTROLLER_ID, NO_CONTROLLER_ID);
 
         // This field only exists in v2+
-        if (struct.hasField(CLUSTER_ID_KEY_NAME)) {
-            this.clusterId = struct.getString(CLUSTER_ID_KEY_NAME);
-        } else {
-            this.clusterId = null;
-        }
+        this.clusterId = struct.getOrElse(CLUSTER_ID, null);
 
         List<TopicMetadata> topicMetadata = new ArrayList<>();
-        Object[] topicInfos = (Object[]) struct.get(TOPIC_METADATA_KEY_NAME);
+        Object[] topicInfos = struct.get(TOPIC_METADATA);
         for (Object topicInfoObj : topicInfos) {
             Struct topicInfo = (Struct) topicInfoObj;
             Errors topicError = Errors.forCode(topicInfo.get(ERROR_CODE));
             String topic = topicInfo.get(TOPIC_NAME);
             // This field only exists in v1+
             // When we can't know if a topic is internal or not in a v0 response we default to false
-            boolean isInternal = topicInfo.hasField(IS_INTERNAL_KEY_NAME) ? topicInfo.getBoolean(IS_INTERNAL_KEY_NAME) : false;
-
+            boolean isInternal = topicInfo.getOrElse(IS_INTERNAL, false);
             List<PartitionMetadata> partitionMetadata = new ArrayList<>();
 
-            Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
+            Object[] partitionInfos = topicInfo.get(PARTITION_METADATA);
             for (Object partitionInfoObj : partitionInfos) {
                 Struct partitionInfo = (Struct) partitionInfoObj;
                 Errors partitionError = Errors.forCode(partitionInfo.get(ERROR_CODE));
                 int partition = partitionInfo.get(PARTITION_ID);
-                int leader = partitionInfo.getInt(LEADER_KEY_NAME);
+                int leader = partitionInfo.get(LEADER);
+                Optional<Integer> leaderEpoch = RequestUtils.getLeaderEpoch(partitionInfo, LEADER_EPOCH);
                 Node leaderNode = leader == -1 ? null : brokers.get(leader);
 
-                Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
+                Object[] replicas = partitionInfo.get(REPLICAS);
                 List<Node> replicaNodes = convertToNodes(brokers, replicas);
 
-                Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
+                Object[] isr = partitionInfo.get(ISR);
                 List<Node> isrNodes = convertToNodes(brokers, isr);
 
-                Object[] offlineReplicas = partitionInfo.hasField(OFFLINE_REPLICAS_KEY_NAME) ?
-                    (Object[]) partitionInfo.get(OFFLINE_REPLICAS_KEY_NAME) : new Object[0];
+                Object[] offlineReplicas = partitionInfo.getOrEmpty(OFFLINE_REPLICAS);
                 List<Node> offlineNodes = convertToNodes(brokers, offlineReplicas);
 
-                partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes, offlineNodes));
+                partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, leaderEpoch,
+                        replicaNodes, isrNodes, offlineNodes));
             }
 
             topicMetadata.add(new TopicMetadata(topicError, topic, isInternal, partitionMetadata));
@@ -354,7 +374,7 @@ public class MetadataResponse extends AbstractResponse {
             if (metadata.error == Errors.NONE) {
                 if (metadata.isInternal)
                     internalTopics.add(metadata.topic);
-                for (PartitionMetadata partitionMetadata : metadata.partitionMetadata)
+                for (PartitionMetadata partitionMetadata : metadata.partitionMetadata) {
                     partitions.add(new PartitionInfo(
                             metadata.topic,
                             partitionMetadata.partition,
@@ -362,6 +382,7 @@ public class MetadataResponse extends AbstractResponse {
                             partitionMetadata.replicas.toArray(new Node[0]),
                             partitionMetadata.isr.toArray(new Node[0]),
                             partitionMetadata.offlineReplicas.toArray(new Node[0])));
+                }
             }
         }
 
@@ -452,6 +473,7 @@ public class MetadataResponse extends AbstractResponse {
         private final Errors error;
         private final int partition;
         private final Node leader;
+        private final Optional<Integer> leaderEpoch;
         private final List<Node> replicas;
         private final List<Node> isr;
         private final List<Node> offlineReplicas;
@@ -459,12 +481,14 @@ public class MetadataResponse extends AbstractResponse {
         public PartitionMetadata(Errors error,
                                  int partition,
                                  Node leader,
+                                 Optional<Integer> leaderEpoch,
                                  List<Node> replicas,
                                  List<Node> isr,
                                  List<Node> offlineReplicas) {
             this.error = error;
             this.partition = partition;
             this.leader = leader;
+            this.leaderEpoch = leaderEpoch;
             this.replicas = replicas;
             this.isr = isr;
             this.offlineReplicas = offlineReplicas;
@@ -482,6 +506,10 @@ public class MetadataResponse extends AbstractResponse {
             return leader == null ? -1 : leader.id();
         }
 
+        public Optional<Integer> leaderEpoch() {
+            return leaderEpoch;
+        }
+
         public Node leader() {
             return leader;
         }
@@ -504,6 +532,7 @@ public class MetadataResponse extends AbstractResponse {
                     ", error=" + error +
                     ", partition=" + partition +
                     ", leader=" + leader +
+                    ", leaderEpoch=" + leaderEpoch +
                     ", replicas=" + Utils.join(replicas, ",") +
                     ", isr=" + Utils.join(isr, ",") +
                     ", offlineReplicas=" + Utils.join(offlineReplicas, ",") + ')';
@@ -516,61 +545,61 @@ public class MetadataResponse extends AbstractResponse {
         struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> brokerArray = new ArrayList<>();
         for (Node node : brokers) {
-            Struct broker = struct.instance(BROKERS_KEY_NAME);
-            broker.set(NODE_ID_KEY_NAME, node.id());
-            broker.set(HOST_KEY_NAME, node.host());
-            broker.set(PORT_KEY_NAME, node.port());
+            Struct broker = struct.instance(BROKERS);
+            broker.set(NODE_ID, node.id());
+            broker.set(HOST, node.host());
+            broker.set(PORT, node.port());
             // This field only exists in v1+
-            if (broker.hasField(RACK_KEY_NAME))
-                broker.set(RACK_KEY_NAME, node.rack());
+            broker.setIfExists(RACK, node.rack());
             brokerArray.add(broker);
         }
-        struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
+        struct.set(BROKERS, brokerArray.toArray());
 
         // This field only exists in v1+
-        if (struct.hasField(CONTROLLER_ID_KEY_NAME))
-            struct.set(CONTROLLER_ID_KEY_NAME, controller == null ? NO_CONTROLLER_ID : controller.id());
+        struct.setIfExists(CONTROLLER_ID, controller == null ? NO_CONTROLLER_ID : controller.id());
 
         // This field only exists in v2+
-        if (struct.hasField(CLUSTER_ID_KEY_NAME))
-            struct.set(CLUSTER_ID_KEY_NAME, clusterId);
+        struct.setIfExists(CLUSTER_ID, clusterId);
 
         List<Struct> topicMetadataArray = new ArrayList<>(topicMetadata.size());
         for (TopicMetadata metadata : topicMetadata) {
-            Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME);
+            Struct topicData = struct.instance(TOPIC_METADATA);
             topicData.set(TOPIC_NAME, metadata.topic);
             topicData.set(ERROR_CODE, metadata.error.code());
             // This field only exists in v1+
-            if (topicData.hasField(IS_INTERNAL_KEY_NAME))
-                topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal());
+            topicData.setIfExists(IS_INTERNAL, metadata.isInternal());
 
             List<Struct> partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size());
             for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) {
-                Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
+                Struct partitionData = topicData.instance(PARTITION_METADATA);
                 partitionData.set(ERROR_CODE, partitionMetadata.error.code());
                 partitionData.set(PARTITION_ID, partitionMetadata.partition);
-                partitionData.set(LEADER_KEY_NAME, partitionMetadata.leaderId());
+                partitionData.set(LEADER, partitionMetadata.leaderId());
+
+                // Leader epoch exists in v7 forward
+                RequestUtils.setLeaderEpochIfExists(partitionData, LEADER_EPOCH, partitionMetadata.leaderEpoch);
+
                 ArrayList<Integer> replicas = new ArrayList<>(partitionMetadata.replicas.size());
                 for (Node node : partitionMetadata.replicas)
                     replicas.add(node.id());
-                partitionData.set(REPLICAS_KEY_NAME, replicas.toArray());
+                partitionData.set(REPLICAS, replicas.toArray());
                 ArrayList<Integer> isr = new ArrayList<>(partitionMetadata.isr.size());
                 for (Node node : partitionMetadata.isr)
                     isr.add(node.id());
-                partitionData.set(ISR_KEY_NAME, isr.toArray());
-                if (partitionData.hasField(OFFLINE_REPLICAS_KEY_NAME)) {
+                partitionData.set(ISR, isr.toArray());
+                if (partitionData.hasField(OFFLINE_REPLICAS)) {
                     ArrayList<Integer> offlineReplicas = new ArrayList<>(partitionMetadata.offlineReplicas.size());
                     for (Node node : partitionMetadata.offlineReplicas)
                         offlineReplicas.add(node.id());
-                    partitionData.set(OFFLINE_REPLICAS_KEY_NAME, offlineReplicas.toArray());
+                    partitionData.set(OFFLINE_REPLICAS, offlineReplicas.toArray());
                 }
                 partitionMetadataArray.add(partitionData);
 
             }
-            topicData.set(PARTITION_METADATA_KEY_NAME, partitionMetadataArray.toArray());
+            topicData.set(PARTITION_METADATA, partitionMetadataArray.toArray());
             topicMetadataArray.add(topicData);
         }
-        struct.set(TOPIC_METADATA_KEY_NAME, topicMetadataArray.toArray());
+        struct.set(TOPIC_METADATA, topicMetadataArray.toArray());
         return struct;
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
index 8a51e84..fdb5b10 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
@@ -17,10 +17,8 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -31,95 +29,112 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_LEADER_EPOCH;
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_METADATA;
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_OFFSET;
 import static org.apache.kafka.common.protocol.CommonFields.GENERATION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.GROUP_ID;
 import static org.apache.kafka.common.protocol.CommonFields.MEMBER_ID;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
-import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
 
-/**
- * This wrapper supports both v0 and v1 of OffsetCommitRequest.
- */
 public class OffsetCommitRequest extends AbstractRequest {
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String RETENTION_TIME_KEY_NAME = "retention_time";
-
-    // topic level field names
-    private static final String PARTITIONS_KEY_NAME = "partitions";
+    // top level fields
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "Topics to commit offsets");
 
-    // partition level field names
-    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "Partitions to commit offsets");
 
+    // partition level fields
     @Deprecated
-    private static final String TIMESTAMP_KEY_NAME = "timestamp";         // for v0, v1
+    private static final Field.Int64 COMMIT_TIMESTAMP = new Field.Int64("timestamp", "Timestamp of the commit");
+    private static final Field.Int64 RETENTION_TIME = new Field.Int64("retention_time",
+            "Time period in ms to retain the offset.");
 
-    /* Offset commit api */
-    private static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
-            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Message offset to be committed."),
-            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."));
+            COMMITTED_OFFSET,
+            COMMITTED_METADATA);
 
-    private static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(
-            PARTITION_ID,
-            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Message offset to be committed."),
-            new Field(TIMESTAMP_KEY_NAME, INT64, "Timestamp of the commit"),
-            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."));
-
-    private static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(
-            PARTITION_ID,
-            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Message offset to be committed."),
-            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."));
-
-    private static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(
-            TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), "Partitions to commit offsets."));
-
-    private static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), "Partitions to commit offsets."));
-
-    private static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(
-            TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2), "Partitions to commit offsets."));
+            PARTITIONS_V0);
 
     private static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(
             GROUP_ID,
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), "Topics to commit offsets."));
+            TOPICS_V0);
+
+    // V1 adds timestamp and group membership information (generation and memberId)
+    private static final Field PARTITIONS_V1 = PARTITIONS.withFields(
+            PARTITION_ID,
+            COMMITTED_OFFSET,
+            COMMIT_TIMESTAMP,
+            COMMITTED_METADATA);
+
+    private static final Field TOPICS_V1 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V1);
 
     private static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(
             GROUP_ID,
             GENERATION_ID,
             MEMBER_ID,
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), "Topics to commit offsets."));
+            TOPICS_V1);
+
+    // V2 adds retention time
+    private static final Field PARTITIONS_V2 = PARTITIONS.withFields(
+            PARTITION_ID,
+            COMMITTED_OFFSET,
+            COMMITTED_METADATA);
+
+    private static final Field TOPICS_V2 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V2);
 
     private static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(
             GROUP_ID,
             GENERATION_ID,
             MEMBER_ID,
-            new Field(RETENTION_TIME_KEY_NAME, INT64, "Time period in ms to retain the offset."),
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2), "Topics to commit offsets."));
+            RETENTION_TIME,
+            TOPICS_V2);
 
-    /* v3 request is same as v2. Throttle time has been added to response */
+    // V3 adds throttle time
     private static final Schema OFFSET_COMMIT_REQUEST_V3 = OFFSET_COMMIT_REQUEST_V2;
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V4 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema OFFSET_COMMIT_REQUEST_V4 = OFFSET_COMMIT_REQUEST_V3;
 
+    // V5 removes the retention time which is now controlled only by a broker configuration
     private static final Schema OFFSET_COMMIT_REQUEST_V5 = new Schema(
             GROUP_ID,
             GENERATION_ID,
             MEMBER_ID,
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2), "Topics to commit offsets."));
+            TOPICS_V2);
+
+    // V6 adds the leader epoch to the partition data
+    private static final Field PARTITIONS_V6 = PARTITIONS.withFields(
+            PARTITION_ID,
+            COMMITTED_OFFSET,
+            COMMITTED_LEADER_EPOCH,
+            COMMITTED_METADATA);
+
+    private static final Field TOPICS_V6 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V6);
+
+    private static final Schema OFFSET_COMMIT_REQUEST_V6 = new Schema(
+            GROUP_ID,
+            GENERATION_ID,
+            MEMBER_ID,
+            TOPICS_V6);
 
     public static Schema[] schemaVersions() {
         return new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2,
-            OFFSET_COMMIT_REQUEST_V3, OFFSET_COMMIT_REQUEST_V4, OFFSET_COMMIT_REQUEST_V5};
+            OFFSET_COMMIT_REQUEST_V3, OFFSET_COMMIT_REQUEST_V4, OFFSET_COMMIT_REQUEST_V5, OFFSET_COMMIT_REQUEST_V6};
     }
 
     // default values for the current version
@@ -144,25 +159,32 @@ public class OffsetCommitRequest extends AbstractRequest {
 
         public final long offset;
         public final String metadata;
+        public final Optional<Integer> leaderEpoch;
 
-        @Deprecated
-        public PartitionData(long offset, long timestamp, String metadata) {
+        private PartitionData(long offset, Optional<Integer> leaderEpoch, long timestamp, String metadata) {
             this.offset = offset;
+            this.leaderEpoch = leaderEpoch;
             this.timestamp = timestamp;
             this.metadata = metadata;
         }
 
-        public PartitionData(long offset, String metadata) {
-            this(offset, DEFAULT_TIMESTAMP, metadata);
+        @Deprecated
+        public PartitionData(long offset, long timestamp, String metadata) {
+            this(offset, Optional.empty(), timestamp, metadata);
+        }
+
+        public PartitionData(long offset, Optional<Integer> leaderEpoch, String metadata) {
+            this(offset, leaderEpoch, DEFAULT_TIMESTAMP, metadata);
         }
 
         @Override
         public String toString() {
             StringBuilder bld = new StringBuilder();
             bld.append("(timestamp=").append(timestamp).
-                append(", offset=").append(offset).
-                append(", metadata=").append(metadata).
-                append(")");
+                    append(", offset=").append(offset).
+                    append(", leaderEpoch=").append(leaderEpoch).
+                    append(", metadata=").append(metadata).
+                    append(")");
             return bld.toString();
         }
     }
@@ -191,18 +213,12 @@ public class OffsetCommitRequest extends AbstractRequest {
 
         @Override
         public OffsetCommitRequest build(short version) {
-            switch (version) {
-                case 0:
-                    return new OffsetCommitRequest(groupId, DEFAULT_GENERATION_ID, DEFAULT_MEMBER_ID,
-                            DEFAULT_RETENTION_TIME, offsetData, version);
-                case 1:
-                case 2:
-                case 3:
-                case 4:
-                case 5:
-                    return new OffsetCommitRequest(groupId, generationId, memberId, DEFAULT_RETENTION_TIME, offsetData, version);
-                default:
-                    throw new UnsupportedVersionException("Unsupported version " + version);
+            if (version == 0) {
+                return new OffsetCommitRequest(groupId, DEFAULT_GENERATION_ID, DEFAULT_MEMBER_ID,
+                        DEFAULT_RETENTION_TIME, offsetData, version);
+            } else {
+                return new OffsetCommitRequest(groupId, generationId, memberId, DEFAULT_RETENTION_TIME,
+                        offsetData, version);
             }
         }
 
@@ -221,7 +237,7 @@ public class OffsetCommitRequest extends AbstractRequest {
 
     private OffsetCommitRequest(String groupId, int generationId, String memberId, long retentionTime,
                                 Map<TopicPartition, PartitionData> offsetData, short version) {
-        super(version);
+        super(ApiKeys.OFFSET_COMMIT, version);
         this.groupId = groupId;
         this.generationId = generationId;
         this.memberId = memberId;
@@ -230,7 +246,7 @@ public class OffsetCommitRequest extends AbstractRequest {
     }
 
     public OffsetCommitRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.OFFSET_COMMIT, versionId);
 
         groupId = struct.get(GROUP_ID);
 
@@ -239,27 +255,26 @@ public class OffsetCommitRequest extends AbstractRequest {
         memberId = struct.getOrElse(MEMBER_ID, DEFAULT_MEMBER_ID);
 
         // This field only exists in v2
-        if (struct.hasField(RETENTION_TIME_KEY_NAME))
-            retentionTime = struct.getLong(RETENTION_TIME_KEY_NAME);
-        else
-            retentionTime = DEFAULT_RETENTION_TIME;
+        retentionTime = struct.getOrElse(RETENTION_TIME, DEFAULT_RETENTION_TIME);
 
         offsetData = new HashMap<>();
-        for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) {
+        for (Object topicDataObj : struct.get(TOPICS)) {
             Struct topicData = (Struct) topicDataObj;
             String topic = topicData.get(TOPIC_NAME);
-            for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionDataObj : topicData.get(PARTITIONS)) {
                 Struct partitionDataStruct = (Struct) partitionDataObj;
                 int partition = partitionDataStruct.get(PARTITION_ID);
-                long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME);
-                String metadata = partitionDataStruct.getString(METADATA_KEY_NAME);
+                long offset = partitionDataStruct.get(COMMITTED_OFFSET);
+                String metadata = partitionDataStruct.get(COMMITTED_METADATA);
                 PartitionData partitionOffset;
                 // This field only exists in v1
-                if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) {
-                    long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME);
+                if (partitionDataStruct.hasField(COMMIT_TIMESTAMP)) {
+                    long timestamp = partitionDataStruct.get(COMMIT_TIMESTAMP);
                     partitionOffset = new PartitionData(offset, timestamp, metadata);
                 } else {
-                    partitionOffset = new PartitionData(offset, metadata);
+                    Optional<Integer> leaderEpochOpt = RequestUtils.getLeaderEpoch(partitionDataStruct,
+                            COMMITTED_LEADER_EPOCH);
+                    partitionOffset = new PartitionData(offset, leaderEpochOpt, metadata);
                 }
                 offsetData.put(new TopicPartition(topic, partition), partitionOffset);
             }
@@ -272,31 +287,31 @@ public class OffsetCommitRequest extends AbstractRequest {
         Struct struct = new Struct(ApiKeys.OFFSET_COMMIT.requestSchema(version));
         struct.set(GROUP_ID, groupId);
 
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupPartitionDataByTopic(offsetData);
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
-            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            Struct topicData = struct.instance(TOPICS);
             topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                Struct partitionData = topicData.instance(PARTITIONS);
                 partitionData.set(PARTITION_ID, partitionEntry.getKey());
-                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                partitionData.set(COMMITTED_OFFSET, fetchPartitionData.offset);
                 // Only for v1
-                if (partitionData.hasField(TIMESTAMP_KEY_NAME))
-                    partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
-                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+                partitionData.setIfExists(COMMIT_TIMESTAMP, fetchPartitionData.timestamp);
+                // Only for v6
+                RequestUtils.setLeaderEpochIfExists(partitionData, COMMITTED_LEADER_EPOCH, fetchPartitionData.leaderEpoch);
+                partitionData.set(COMMITTED_METADATA, fetchPartitionData.metadata);
                 partitionArray.add(partitionData);
             }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicData.set(PARTITIONS, partitionArray.toArray());
             topicArray.add(topicData);
         }
-        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        struct.set(TOPICS, topicArray.toArray());
         struct.setIfExists(GENERATION_ID, generationId);
         struct.setIfExists(MEMBER_ID, memberId);
-        if (struct.hasField(RETENTION_TIME_KEY_NAME))
-            struct.set(RETENTION_TIME_KEY_NAME, retentionTime);
+        struct.setIfExists(RETENTION_TIME, retentionTime);
         return struct;
     }
 
@@ -316,6 +331,7 @@ public class OffsetCommitRequest extends AbstractRequest {
             case 3:
             case 4:
             case 5:
+            case 6:
                 return new OffsetCommitResponse(throttleTimeMs, responseData);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
@@ -335,6 +351,7 @@ public class OffsetCommitRequest extends AbstractRequest {
         return memberId;
     }
 
+    @Deprecated
     public long retentionTime() {
         return retentionTime;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
index c79bc57..4d724a3 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -36,60 +35,64 @@ import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 
+/**
+ * Possible error codes:
+ *
+ * UNKNOWN_TOPIC_OR_PARTITION (3)
+ * REQUEST_TIMED_OUT (7)
+ * OFFSET_METADATA_TOO_LARGE (12)
+ * COORDINATOR_LOAD_IN_PROGRESS (14)
+ * GROUP_COORDINATOR_NOT_AVAILABLE (15)
+ * NOT_COORDINATOR (16)
+ * ILLEGAL_GENERATION (22)
+ * UNKNOWN_MEMBER_ID (25)
+ * REBALANCE_IN_PROGRESS (27)
+ * INVALID_COMMIT_OFFSET_SIZE (28)
+ * TOPIC_AUTHORIZATION_FAILED (29)
+ * GROUP_AUTHORIZATION_FAILED (30)
+ */
 public class OffsetCommitResponse extends AbstractResponse {
-
-    private static final String RESPONSES_KEY_NAME = "responses";
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("responses",
+            "Responses by topic for committed partitions");
 
     // topic level fields
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    /**
-     * Possible error codes:
-     *
-     * UNKNOWN_TOPIC_OR_PARTITION (3)
-     * REQUEST_TIMED_OUT (7)
-     * OFFSET_METADATA_TOO_LARGE (12)
-     * COORDINATOR_LOAD_IN_PROGRESS (14)
-     * GROUP_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR (16)
-     * ILLEGAL_GENERATION (22)
-     * UNKNOWN_MEMBER_ID (25)
-     * REBALANCE_IN_PROGRESS (27)
-     * INVALID_COMMIT_OFFSET_SIZE (28)
-     * TOPIC_AUTHORIZATION_FAILED (29)
-     * GROUP_AUTHORIZATION_FAILED (30)
-     */
-
-    private static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partition_responses",
+            "Responses for committed partitions");
+
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
             ERROR_CODE);
 
-    private static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
+            PARTITIONS_V0);
 
     private static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+            TOPICS_V0);
 
-
-    /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
+    // V1 adds timestamp and group membership information (generation and memberId) to the request
     private static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
+
+    // V2 adds retention time to the request
     private static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
 
+    // V3 adds throttle time
     private static final Schema OFFSET_COMMIT_RESPONSE_V3 = new Schema(
             THROTTLE_TIME_MS,
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+            TOPICS_V0);
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V4 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema OFFSET_COMMIT_RESPONSE_V4 = OFFSET_COMMIT_RESPONSE_V3;
 
+    // V5 removes retention time from the request
     private static final Schema OFFSET_COMMIT_RESPONSE_V5 = OFFSET_COMMIT_RESPONSE_V4;
 
+    // V6 adds leader epoch to the request
+    private static final Schema OFFSET_COMMIT_RESPONSE_V6 = OFFSET_COMMIT_RESPONSE_V5;
+
     public static Schema[] schemaVersions() {
         return new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2,
-            OFFSET_COMMIT_RESPONSE_V3, OFFSET_COMMIT_RESPONSE_V4, OFFSET_COMMIT_RESPONSE_V5};
+            OFFSET_COMMIT_RESPONSE_V3, OFFSET_COMMIT_RESPONSE_V4, OFFSET_COMMIT_RESPONSE_V5, OFFSET_COMMIT_RESPONSE_V6};
     }
 
     private final Map<TopicPartition, Errors> responseData;
@@ -107,10 +110,10 @@ public class OffsetCommitResponse extends AbstractResponse {
     public OffsetCommitResponse(Struct struct) {
         this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         responseData = new HashMap<>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+        for (Object topicResponseObj : struct.get(TOPICS)) {
             Struct topicResponse = (Struct) topicResponseObj;
             String topic = topicResponse.get(TOPIC_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionResponseObj : topicResponse.get(PARTITIONS)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
                 int partition = partitionResponse.get(PARTITION_ID);
                 Errors error = Errors.forCode(partitionResponse.get(ERROR_CODE));
@@ -124,22 +127,22 @@ public class OffsetCommitResponse extends AbstractResponse {
         Struct struct = new Struct(ApiKeys.OFFSET_COMMIT.responseSchema(version));
         struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
-        Map<String, Map<Integer, Errors>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+        Map<String, Map<Integer, Errors>> topicsData = CollectionUtils.groupPartitionDataByTopic(responseData);
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Errors>> entries: topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            Struct topicData = struct.instance(TOPICS);
             topicData.set(TOPIC_NAME, entries.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, Errors> partitionEntry : entries.getValue().entrySet()) {
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                Struct partitionData = topicData.instance(PARTITIONS);
                 partitionData.set(PARTITION_ID, partitionEntry.getKey());
                 partitionData.set(ERROR_CODE, partitionEntry.getValue().code());
                 partitionArray.add(partitionData);
             }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicData.set(PARTITIONS, partitionArray.toArray());
             topicArray.add(topicData);
         }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        struct.set(TOPICS, topicArray.toArray());
 
         return struct;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
index e90c6eb..d2f5c88 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
@@ -20,7 +20,6 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -32,16 +31,20 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
 import static org.apache.kafka.common.protocol.CommonFields.GROUP_ID;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 
 public class OffsetFetchRequest extends AbstractRequest {
-    private static final String TOPICS_KEY_NAME = "topics";
+    // top level fields
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "Topics to fetch offsets. If the topic array is null fetch offsets for all topics.");
 
-    // topic level field names
-    private static final String PARTITIONS_KEY_NAME = "partitions";
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "Partitions to fetch offsets.");
 
     /*
      * Wire formats of version 0 and 1 are the same, but with different functionality.
@@ -54,32 +57,41 @@ public class OffsetFetchRequest extends AbstractRequest {
      * a 'null' is passed instead of a list of specific topic partitions. It also returns a top level error code
      * for group or coordinator level errors.
      */
-    private static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID);
 
-    private static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(
+    private static final Field TOPICS_V0 = TOPICS.withFields("Topics to fetch offsets.",
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), "Partitions to fetch offsets."));
+            PARTITIONS_V0);
 
     private static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(
             GROUP_ID,
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0), "Topics to fetch offsets."));
+            TOPICS_V0);
 
+    // V1 begins support for fetching offsets from the internal __consumer_offsets topic
     private static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
 
+    // V2 adds top-level error code to the response as well as allowing a null offset array to indicate fetch
+    // of all committed offsets for a group
+    private static final Field TOPICS_V2 = TOPICS.nullableWithFields(
+            TOPIC_NAME,
+            PARTITIONS_V0);
     private static final Schema OFFSET_FETCH_REQUEST_V2 = new Schema(
             GROUP_ID,
-            new Field(TOPICS_KEY_NAME, ArrayOf.nullable(OFFSET_FETCH_REQUEST_TOPIC_V0), "Topics to fetch offsets. If the " +
-                    "topic array is null fetch offsets for all topics."));
+            TOPICS_V2);
 
-    /* v3 request is the same as v2. Throttle time has been added to v3 response */
+    // V3 request is the same as v2. Throttle time has been added to v3 response
     private static final Schema OFFSET_FETCH_REQUEST_V3 = OFFSET_FETCH_REQUEST_V2;
 
+    // V4 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema OFFSET_FETCH_REQUEST_V4 = OFFSET_FETCH_REQUEST_V3;
 
+    // V5 adds the leader epoch of the committed offset in the response
+    private static final Schema OFFSET_FETCH_REQUEST_V5 = OFFSET_FETCH_REQUEST_V4;
+
     public static Schema[] schemaVersions() {
         return new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1, OFFSET_FETCH_REQUEST_V2,
-            OFFSET_FETCH_REQUEST_V3, OFFSET_FETCH_REQUEST_V4};
+            OFFSET_FETCH_REQUEST_V3, OFFSET_FETCH_REQUEST_V4, OFFSET_FETCH_REQUEST_V5};
     }
 
     public static class Builder extends AbstractRequest.Builder<OffsetFetchRequest> {
@@ -128,23 +140,22 @@ public class OffsetFetchRequest extends AbstractRequest {
         return new OffsetFetchRequest.Builder(groupId, null).build((short) 2);
     }
 
-    // v0, v1, and v2 have the same fields.
     private OffsetFetchRequest(String groupId, List<TopicPartition> partitions, short version) {
-        super(version);
+        super(ApiKeys.OFFSET_FETCH, version);
         this.groupId = groupId;
         this.partitions = partitions;
     }
 
     public OffsetFetchRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.OFFSET_FETCH, version);
 
-        Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
+        Object[] topicArray = struct.get(TOPICS);
         if (topicArray != null) {
             partitions = new ArrayList<>();
-            for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            for (Object topicResponseObj : topicArray) {
                 Struct topicResponse = (Struct) topicResponseObj;
                 String topic = topicResponse.get(TOPIC_NAME);
-                for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                for (Object partitionResponseObj : topicResponse.get(PARTITIONS)) {
                     Struct partitionResponse = (Struct) partitionResponseObj;
                     int partition = partitionResponse.get(PARTITION_ID);
                     partitions.add(new TopicPartition(topic, partition));
@@ -166,12 +177,14 @@ public class OffsetFetchRequest extends AbstractRequest {
 
         Map<TopicPartition, OffsetFetchResponse.PartitionData> responsePartitions = new HashMap<>();
         if (versionId < 2) {
-            for (TopicPartition partition : this.partitions) {
-                responsePartitions.put(partition, new OffsetFetchResponse.PartitionData(
-                        OffsetFetchResponse.INVALID_OFFSET,
-                        OffsetFetchResponse.NO_METADATA,
-                        error));
-            }
+            OffsetFetchResponse.PartitionData partitionError = new OffsetFetchResponse.PartitionData(
+                    OffsetFetchResponse.INVALID_OFFSET,
+                    Optional.empty(),
+                    OffsetFetchResponse.NO_METADATA,
+                    error);
+
+            for (TopicPartition partition : this.partitions)
+                responsePartitions.put(partition, partitionError);
         }
 
         switch (versionId) {
@@ -181,6 +194,7 @@ public class OffsetFetchRequest extends AbstractRequest {
                 return new OffsetFetchResponse(error, responsePartitions);
             case 3:
             case 4:
+            case 5:
                 return new OffsetFetchResponse(throttleTimeMs, error, responsePartitions);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
@@ -214,25 +228,26 @@ public class OffsetFetchRequest extends AbstractRequest {
         Struct struct = new Struct(ApiKeys.OFFSET_FETCH.requestSchema(version()));
         struct.set(GROUP_ID, groupId);
         if (partitions != null) {
-            Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
+            Map<String, List<Integer>> topicsData = CollectionUtils.groupPartitionsByTopic(partitions);
 
             List<Struct> topicArray = new ArrayList<>();
             for (Map.Entry<String, List<Integer>> entries : topicsData.entrySet()) {
-                Struct topicData = struct.instance(TOPICS_KEY_NAME);
+                Struct topicData = struct.instance(TOPICS);
                 topicData.set(TOPIC_NAME, entries.getKey());
                 List<Struct> partitionArray = new ArrayList<>();
                 for (Integer partitionId : entries.getValue()) {
-                    Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                    Struct partitionData = topicData.instance(PARTITIONS);
                     partitionData.set(PARTITION_ID, partitionId);
                     partitionArray.add(partitionData);
                 }
-                topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+                topicData.set(PARTITIONS, partitionArray.toArray());
                 topicArray.add(topicData);
             }
-            struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+            struct.set(TOPICS, topicArray.toArray());
         } else
-            struct.set(TOPICS_KEY_NAME, null);
+            struct.set(TOPICS, null);
 
         return struct;
     }
+
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
index 613695b..2022bb7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -32,79 +31,94 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_LEADER_EPOCH;
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_METADATA;
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_OFFSET;
 import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
-import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
 
+/**
+ * Possible error codes:
+ *
+ * - Partition errors:
+ *   - UNKNOWN_TOPIC_OR_PARTITION (3)
+ *
+ * - Group or coordinator errors:
+ *   - COORDINATOR_LOAD_IN_PROGRESS (14)
+ *   - COORDINATOR_NOT_AVAILABLE (15)
+ *   - NOT_COORDINATOR (16)
+ *   - GROUP_AUTHORIZATION_FAILED (30)
+ */
 public class OffsetFetchResponse extends AbstractResponse {
-
-    private static final String RESPONSES_KEY_NAME = "responses";
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("responses",
+            "Responses by topic for fetched offsets");
 
     // topic level fields
-    private static final String PARTITIONS_KEY_NAME = "partition_responses";
-
-    // partition level fields
-    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partition_responses",
+            "Responses by partition for fetched offsets");
 
-    private static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
-            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Last committed message offset."),
-            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."),
+            COMMITTED_OFFSET,
+            COMMITTED_METADATA,
             ERROR_CODE);
 
-    private static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
+            PARTITIONS_V0);
 
     private static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
+            TOPICS_V0);
 
+    // V1 begins support for fetching offsets from the internal __consumer_offsets topic
     private static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
 
+    // V2 adds top-level error code
     private static final Schema OFFSET_FETCH_RESPONSE_V2 = new Schema(
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)),
+            TOPICS_V0,
             ERROR_CODE);
 
-    /* v3 request is the same as v2. Throttle time has been added to v3 response */
+    // V3 request includes throttle time
     private static final Schema OFFSET_FETCH_RESPONSE_V3 = new Schema(
             THROTTLE_TIME_MS,
-            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)),
+            TOPICS_V0,
             ERROR_CODE);
 
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+    // V4 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema OFFSET_FETCH_RESPONSE_V4 = OFFSET_FETCH_RESPONSE_V3;
 
+    // V5 adds the leader epoch to the committed offset
+    private static final Field PARTITIONS_V5 = PARTITIONS.withFields(
+            PARTITION_ID,
+            COMMITTED_OFFSET,
+            COMMITTED_LEADER_EPOCH,
+            COMMITTED_METADATA,
+            ERROR_CODE);
+
+    private static final Field TOPICS_V5 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V5);
+
+    private static final Schema OFFSET_FETCH_RESPONSE_V5 = new Schema(
+            THROTTLE_TIME_MS,
+            TOPICS_V5,
+            ERROR_CODE);
+
     public static Schema[] schemaVersions() {
         return new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1, OFFSET_FETCH_RESPONSE_V2,
-            OFFSET_FETCH_RESPONSE_V3, OFFSET_FETCH_RESPONSE_V4};
+            OFFSET_FETCH_RESPONSE_V3, OFFSET_FETCH_RESPONSE_V4, OFFSET_FETCH_RESPONSE_V5};
     }
 
     public static final long INVALID_OFFSET = -1L;
     public static final String NO_METADATA = "";
-    public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET, NO_METADATA,
-            Errors.UNKNOWN_TOPIC_OR_PARTITION);
-    public static final PartitionData UNAUTHORIZED_PARTITION = new PartitionData(INVALID_OFFSET, NO_METADATA,
-            Errors.TOPIC_AUTHORIZATION_FAILED);
-
-    /**
-     * Possible error codes:
-     *
-     * - Partition errors:
-     *   - UNKNOWN_TOPIC_OR_PARTITION (3)
-     *
-     * - Group or coordinator errors:
-     *   - COORDINATOR_LOAD_IN_PROGRESS (14)
-     *   - COORDINATOR_NOT_AVAILABLE (15)
-     *   - NOT_COORDINATOR (16)
-     *   - GROUP_AUTHORIZATION_FAILED (30)
-     */
+    public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET,
+            Optional.empty(), NO_METADATA, Errors.UNKNOWN_TOPIC_OR_PARTITION);
+    public static final PartitionData UNAUTHORIZED_PARTITION = new PartitionData(INVALID_OFFSET,
+            Optional.empty(), NO_METADATA, Errors.TOPIC_AUTHORIZATION_FAILED);
 
     private static final List<Errors> PARTITION_ERRORS = Collections.singletonList(Errors.UNKNOWN_TOPIC_OR_PARTITION);
 
@@ -116,9 +130,14 @@ public class OffsetFetchResponse extends AbstractResponse {
         public final long offset;
         public final String metadata;
         public final Errors error;
+        public final Optional<Integer> leaderEpoch;
 
-        public PartitionData(long offset, String metadata, Errors error) {
+        public PartitionData(long offset,
+                             Optional<Integer> leaderEpoch,
+                             String metadata,
+                             Errors error) {
             this.offset = offset;
+            this.leaderEpoch = leaderEpoch;
             this.metadata = metadata;
             this.error = error;
         }
@@ -153,18 +172,21 @@ public class OffsetFetchResponse extends AbstractResponse {
         this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         Errors topLevelError = Errors.NONE;
         this.responseData = new HashMap<>();
-        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+        for (Object topicResponseObj : struct.get(TOPICS)) {
             Struct topicResponse = (Struct) topicResponseObj;
             String topic = topicResponse.get(TOPIC_NAME);
-            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionResponseObj : topicResponse.get(PARTITIONS)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
                 int partition = partitionResponse.get(PARTITION_ID);
-                long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
-                String metadata = partitionResponse.getString(METADATA_KEY_NAME);
+                long offset = partitionResponse.get(COMMITTED_OFFSET);
+                String metadata = partitionResponse.get(COMMITTED_METADATA);
+                Optional<Integer> leaderEpochOpt = RequestUtils.getLeaderEpoch(partitionResponse, COMMITTED_LEADER_EPOCH);
+
                 Errors error = Errors.forCode(partitionResponse.get(ERROR_CODE));
                 if (error != Errors.NONE && !PARTITION_ERRORS.contains(error))
                     topLevelError = error;
-                PartitionData partitionData = new PartitionData(offset, metadata, error);
+
+                PartitionData partitionData = new PartitionData(offset, leaderEpochOpt, metadata, error);
                 this.responseData.put(new TopicPartition(topic, partition), partitionData);
             }
         }
@@ -215,25 +237,26 @@ public class OffsetFetchResponse extends AbstractResponse {
         Struct struct = new Struct(ApiKeys.OFFSET_FETCH.responseSchema(version));
         struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
-        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupPartitionDataByTopic(responseData);
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionData>> entries : topicsData.entrySet()) {
-            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            Struct topicData = struct.instance(TOPICS);
             topicData.set(TOPIC_NAME, entries.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : entries.getValue().entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
-                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                Struct partitionData = topicData.instance(PARTITIONS);
                 partitionData.set(PARTITION_ID, partitionEntry.getKey());
-                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
-                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+                partitionData.set(COMMITTED_OFFSET, fetchPartitionData.offset);
+                RequestUtils.setLeaderEpochIfExists(partitionData, COMMITTED_LEADER_EPOCH, fetchPartitionData.leaderEpoch);
+                partitionData.set(COMMITTED_METADATA, fetchPartitionData.metadata);
                 partitionData.set(ERROR_CODE, fetchPartitionData.error.code());
                 partitionArray.add(partitionData);
             }
-            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicData.set(PARTITIONS, partitionArray.toArray());
             topicArray.add(topicData);
         }
-        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        struct.set(TOPICS, topicArray.toArray());
 
         if (version > 1)
             struct.set(ERROR_CODE, this.error.code());
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
index 651416d..9de5d02 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -30,53 +29,69 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
+import static org.apache.kafka.common.protocol.CommonFields.CURRENT_LEADER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.types.Type.INT32;
 
 public class OffsetsForLeaderEpochRequest extends AbstractRequest {
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String LEADER_EPOCH = "leader_epoch";
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "An array of topics to get epochs for");
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "An array of partitions to get epochs for");
 
-    /* Offsets for Leader Epoch api */
-    private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_PARTITION_V0 = new Schema(
+    private static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch",
+            "The epoch to lookup an offset for.");
+
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
-            new Field(LEADER_EPOCH, INT32, "The epoch"));
-    private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_TOPIC_V0 = new Schema(
+            LEADER_EPOCH);
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_REQUEST_PARTITION_V0)));
+            PARTITIONS_V0);
     private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_V0 = new Schema(
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_REQUEST_TOPIC_V0), "An array of topics to get epochs for"));
+            TOPICS_V0);
 
-    /* v1 request is the same as v0. Per-partition leader epoch has been added to response */
+    // V1 request is the same as v0. Per-partition leader epoch has been added to response
     private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_V1 = OFFSET_FOR_LEADER_EPOCH_REQUEST_V0;
 
+    // V2 adds the current leader epoch to support fencing
+    private static final Field PARTITIONS_V2 = PARTITIONS.withFields(
+            PARTITION_ID,
+            CURRENT_LEADER_EPOCH,
+            LEADER_EPOCH);
+    private static final Field TOPICS_V2 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V2);
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_V2 = new Schema(
+            TOPICS_V2);
+
     public static Schema[] schemaVersions() {
-        return new Schema[]{OFFSET_FOR_LEADER_EPOCH_REQUEST_V0, OFFSET_FOR_LEADER_EPOCH_REQUEST_V1};
+        return new Schema[]{OFFSET_FOR_LEADER_EPOCH_REQUEST_V0, OFFSET_FOR_LEADER_EPOCH_REQUEST_V1,
+            OFFSET_FOR_LEADER_EPOCH_REQUEST_V2};
     }
 
-    private Map<TopicPartition, Integer> epochsByPartition;
+    private Map<TopicPartition, PartitionData> epochsByPartition;
 
-    public Map<TopicPartition, Integer> epochsByTopicPartition() {
+    public Map<TopicPartition, PartitionData> epochsByTopicPartition() {
         return epochsByPartition;
     }
 
     public static class Builder extends AbstractRequest.Builder<OffsetsForLeaderEpochRequest> {
-        private Map<TopicPartition, Integer> epochsByPartition = new HashMap<>();
+        private Map<TopicPartition, PartitionData> epochsByPartition;
 
         public Builder(short version) {
-            super(ApiKeys.OFFSET_FOR_LEADER_EPOCH, version);
+            this(version, new HashMap<>());
         }
 
-        public Builder(short version, Map<TopicPartition, Integer> epochsByPartition) {
+        public Builder(short version, Map<TopicPartition, PartitionData> epochsByPartition) {
             super(ApiKeys.OFFSET_FOR_LEADER_EPOCH, version);
             this.epochsByPartition = epochsByPartition;
         }
 
-        public Builder add(TopicPartition topicPartition, Integer epoch) {
-            epochsByPartition.put(topicPartition, epoch);
+        public Builder add(TopicPartition topicPartition, Optional<Integer> currentEpoch, int leaderEpoch) {
+            epochsByPartition.put(topicPartition, new PartitionData(currentEpoch, leaderEpoch));
             return this;
         }
 
@@ -99,23 +114,24 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
         }
     }
 
-    public OffsetsForLeaderEpochRequest(Map<TopicPartition, Integer> epochsByPartition, short version) {
-        super(version);
+    public OffsetsForLeaderEpochRequest(Map<TopicPartition, PartitionData> epochsByPartition, short version) {
+        super(ApiKeys.OFFSET_FOR_LEADER_EPOCH, version);
         this.epochsByPartition = epochsByPartition;
     }
 
     public OffsetsForLeaderEpochRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.OFFSET_FOR_LEADER_EPOCH, version);
         epochsByPartition = new HashMap<>();
-        for (Object topicAndEpochsObj : struct.getArray(TOPICS_KEY_NAME)) {
+        for (Object topicAndEpochsObj : struct.get(TOPICS)) {
             Struct topicAndEpochs = (Struct) topicAndEpochsObj;
             String topic = topicAndEpochs.get(TOPIC_NAME);
-            for (Object partitionAndEpochObj : topicAndEpochs.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionAndEpochObj : topicAndEpochs.get(PARTITIONS)) {
                 Struct partitionAndEpoch = (Struct) partitionAndEpochObj;
                 int partitionId = partitionAndEpoch.get(PARTITION_ID);
-                int epoch = partitionAndEpoch.getInt(LEADER_EPOCH);
+                int leaderEpoch = partitionAndEpoch.get(LEADER_EPOCH);
+                Optional<Integer> currentEpoch = RequestUtils.getLeaderEpoch(partitionAndEpoch, CURRENT_LEADER_EPOCH);
                 TopicPartition tp = new TopicPartition(topic, partitionId);
-                epochsByPartition.put(tp, epoch);
+                epochsByPartition.put(tp, new PartitionData(currentEpoch, leaderEpoch));
             }
         }
     }
@@ -128,23 +144,28 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
     protected Struct toStruct() {
         Struct requestStruct = new Struct(ApiKeys.OFFSET_FOR_LEADER_EPOCH.requestSchema(version()));
 
-        Map<String, Map<Integer, Integer>> topicsToPartitionEpochs = CollectionUtils.groupDataByTopic(epochsByPartition);
+        Map<String, Map<Integer, PartitionData>> topicsToPartitionEpochs = CollectionUtils.groupPartitionDataByTopic(epochsByPartition);
 
         List<Struct> topics = new ArrayList<>();
-        for (Map.Entry<String, Map<Integer, Integer>> topicToEpochs : topicsToPartitionEpochs.entrySet()) {
-            Struct topicsStruct = requestStruct.instance(TOPICS_KEY_NAME);
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicToEpochs : topicsToPartitionEpochs.entrySet()) {
+            Struct topicsStruct = requestStruct.instance(TOPICS);
             topicsStruct.set(TOPIC_NAME, topicToEpochs.getKey());
             List<Struct> partitions = new ArrayList<>();
-            for (Map.Entry<Integer, Integer> partitionEpoch : topicToEpochs.getValue().entrySet()) {
-                Struct partitionStruct = topicsStruct.instance(PARTITIONS_KEY_NAME);
+            for (Map.Entry<Integer, PartitionData> partitionEpoch : topicToEpochs.getValue().entrySet()) {
+                Struct partitionStruct = topicsStruct.instance(PARTITIONS);
                 partitionStruct.set(PARTITION_ID, partitionEpoch.getKey());
-                partitionStruct.set(LEADER_EPOCH, partitionEpoch.getValue());
+
+                PartitionData partitionData = partitionEpoch.getValue();
+                partitionStruct.set(LEADER_EPOCH, partitionData.leaderEpoch);
+
+                // Current leader epoch introduced in v2
+                RequestUtils.setLeaderEpochIfExists(partitionStruct, CURRENT_LEADER_EPOCH, partitionData.currentLeaderEpoch);
                 partitions.add(partitionStruct);
             }
-            topicsStruct.set(PARTITIONS_KEY_NAME, partitions.toArray());
+            topicsStruct.set(PARTITIONS, partitions.toArray());
             topics.add(topicsStruct);
         }
-        requestStruct.set(TOPICS_KEY_NAME, topics.toArray());
+        requestStruct.set(TOPICS, topics.toArray());
         return requestStruct;
     }
 
@@ -158,4 +179,15 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
         }
         return new OffsetsForLeaderEpochResponse(errorResponse);
     }
+
+    public static class PartitionData {
+        public final Optional<Integer> currentLeaderEpoch;
+        public final int leaderEpoch;
+
+        public PartitionData(Optional<Integer> currentLeaderEpoch, int leaderEpoch) {
+            this.currentLeaderEpoch = currentLeaderEpoch;
+            this.leaderEpoch = leaderEpoch;
+        }
+
+    }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
index 4da8767..324a2ed 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -33,59 +32,61 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.LEADER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
-import static org.apache.kafka.common.protocol.CommonFields.LEADER_EPOCH;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
 
 public class OffsetsForLeaderEpochResponse extends AbstractResponse {
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String END_OFFSET_KEY_NAME = "end_offset";
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "An array of topics for which we have leader offsets for some requested partition leader epoch");
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "An array of offsets by partition");
+    private static final Field.Int64 END_OFFSET = new Field.Int64("end_offset", "The end offset");
 
-    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V0 = new Schema(
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             ERROR_CODE,
             PARTITION_ID,
-            new Field(END_OFFSET_KEY_NAME, INT64, "The end offset"));
-    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V0 = new Schema(
+            END_OFFSET);
+    private static final Field TOPICS_V0 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V0)));
+            PARTITIONS_V0);
     private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0 = new Schema(
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V0),
-                    "An array of topics for which we have leader offsets for some requested Partition Leader Epoch"));
+            TOPICS_V0);
 
-    // OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V1 added a per-partition leader epoch field,
-    // which specifies which leader epoch the end offset belongs to
-    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V1 = new Schema(
+    // V1 added a per-partition leader epoch field which specifies which leader epoch the end offset belongs to
+    private static final Field PARTITIONS_V1 = PARTITIONS.withFields(
             ERROR_CODE,
             PARTITION_ID,
             LEADER_EPOCH,
-            new Field(END_OFFSET_KEY_NAME, INT64, "The end offset"));
-    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V1 = new Schema(
+            END_OFFSET);
+    private static final Field TOPICS_V1 = TOPICS.withFields(
             TOPIC_NAME,
-            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V1)));
+            PARTITIONS_V1);
     private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_V1 = new Schema(
-            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V1),
-                  "An array of topics for which we have leader offsets for some requested Partition Leader Epoch"));
+            TOPICS_V1);
+
+    // V2 bumped for addition of current leader epoch to the request schema.
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_V2 = OFFSET_FOR_LEADER_EPOCH_RESPONSE_V1;
 
     public static Schema[] schemaVersions() {
-        return new Schema[]{OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0, OFFSET_FOR_LEADER_EPOCH_RESPONSE_V1};
+        return new Schema[]{OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0, OFFSET_FOR_LEADER_EPOCH_RESPONSE_V1,
+            OFFSET_FOR_LEADER_EPOCH_RESPONSE_V2};
     }
 
     private Map<TopicPartition, EpochEndOffset> epochEndOffsetsByPartition;
 
     public OffsetsForLeaderEpochResponse(Struct struct) {
         epochEndOffsetsByPartition = new HashMap<>();
-        for (Object topicAndEpocsObj : struct.getArray(TOPICS_KEY_NAME)) {
+        for (Object topicAndEpocsObj : struct.get(TOPICS)) {
             Struct topicAndEpochs = (Struct) topicAndEpocsObj;
             String topic = topicAndEpochs.get(TOPIC_NAME);
-            for (Object partitionAndEpochObj : topicAndEpochs.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionAndEpochObj : topicAndEpochs.get(PARTITIONS)) {
                 Struct partitionAndEpoch = (Struct) partitionAndEpochObj;
                 Errors error = Errors.forCode(partitionAndEpoch.get(ERROR_CODE));
                 int partitionId = partitionAndEpoch.get(PARTITION_ID);
                 TopicPartition tp = new TopicPartition(topic, partitionId);
                 int leaderEpoch = partitionAndEpoch.getOrElse(LEADER_EPOCH, RecordBatch.NO_PARTITION_LEADER_EPOCH);
-                long endOffset = partitionAndEpoch.getLong(END_OFFSET_KEY_NAME);
+                long endOffset = partitionAndEpoch.get(END_OFFSET);
                 epochEndOffsetsByPartition.put(tp, new EpochEndOffset(error, leaderEpoch, endOffset));
             }
         }
@@ -115,26 +116,26 @@ public class OffsetsForLeaderEpochResponse extends AbstractResponse {
     protected Struct toStruct(short version) {
         Struct responseStruct = new Struct(ApiKeys.OFFSET_FOR_LEADER_EPOCH.responseSchema(version));
 
-        Map<String, Map<Integer, EpochEndOffset>> endOffsetsByTopic = CollectionUtils.groupDataByTopic(epochEndOffsetsByPartition);
+        Map<String, Map<Integer, EpochEndOffset>> endOffsetsByTopic = CollectionUtils.groupPartitionDataByTopic(epochEndOffsetsByPartition);
 
         List<Struct> topics = new ArrayList<>(endOffsetsByTopic.size());
         for (Map.Entry<String, Map<Integer, EpochEndOffset>> topicToPartitionEpochs : endOffsetsByTopic.entrySet()) {
-            Struct topicStruct = responseStruct.instance(TOPICS_KEY_NAME);
+            Struct topicStruct = responseStruct.instance(TOPICS);
             topicStruct.set(TOPIC_NAME, topicToPartitionEpochs.getKey());
             Map<Integer, EpochEndOffset> partitionEpochs = topicToPartitionEpochs.getValue();
             List<Struct> partitions = new ArrayList<>();
             for (Map.Entry<Integer, EpochEndOffset> partitionEndOffset : partitionEpochs.entrySet()) {
-                Struct partitionStruct = topicStruct.instance(PARTITIONS_KEY_NAME);
+                Struct partitionStruct = topicStruct.instance(PARTITIONS);
                 partitionStruct.set(ERROR_CODE, partitionEndOffset.getValue().error().code());
                 partitionStruct.set(PARTITION_ID, partitionEndOffset.getKey());
                 partitionStruct.setIfExists(LEADER_EPOCH, partitionEndOffset.getValue().leaderEpoch());
-                partitionStruct.set(END_OFFSET_KEY_NAME, partitionEndOffset.getValue().endOffset());
+                partitionStruct.set(END_OFFSET, partitionEndOffset.getValue().endOffset());
                 partitions.add(partitionStruct);
             }
-            topicStruct.set(PARTITIONS_KEY_NAME, partitions.toArray());
+            topicStruct.set(PARTITIONS, partitions.toArray());
             topics.add(topicStruct);
         }
-        responseStruct.set(TOPICS_KEY_NAME, topics.toArray());
+        responseStruct.set(TOPICS, topics.toArray());
         return responseStruct;
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
index 67745cb..4f1d766 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
@@ -196,7 +196,7 @@ public class ProduceRequest extends AbstractRequest {
     private boolean idempotent = false;
 
     private ProduceRequest(short version, short acks, int timeout, Map<TopicPartition, MemoryRecords> partitionRecords, String transactionalId) {
-        super(version);
+        super(ApiKeys.PRODUCE, version);
         this.acks = acks;
         this.timeout = timeout;
 
@@ -216,7 +216,7 @@ public class ProduceRequest extends AbstractRequest {
     }
 
     public ProduceRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.PRODUCE, version);
         partitionRecords = new HashMap<>();
         for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
             Struct topicData = (Struct) topicDataObj;
@@ -268,7 +268,7 @@ public class ProduceRequest extends AbstractRequest {
         Map<TopicPartition, MemoryRecords> partitionRecords = partitionRecordsOrFail();
         short version = version();
         Struct struct = new Struct(ApiKeys.PRODUCE.requestSchema(version));
-        Map<String, Map<Integer, MemoryRecords>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
+        Map<String, Map<Integer, MemoryRecords>> recordsByTopic = CollectionUtils.groupPartitionDataByTopic(partitionRecords);
         struct.set(ACKS_KEY_NAME, acks);
         struct.set(TIMEOUT_KEY_NAME, timeout);
         struct.setIfExists(NULLABLE_TRANSACTIONAL_ID, transactionalId);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
index 467c980..fb15813 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
@@ -196,7 +196,7 @@ public class ProduceResponse extends AbstractResponse {
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.PRODUCE.responseSchema(version));
 
-        Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupDataByTopic(responses);
+        Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupPartitionDataByTopic(responses);
         List<Struct> topicDatas = new ArrayList<>(responseByTopic.size());
         for (Map.Entry<String, Map<Integer, PartitionResponse>> entry : responseByTopic.entrySet()) {
             Struct topicData = struct.instance(RESPONSES_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java
index 57c3100..d73561a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java
@@ -44,14 +44,14 @@ public class RenewDelegationTokenRequest extends AbstractRequest {
     public static final Schema TOKEN_RENEW_REQUEST_V1 = TOKEN_RENEW_REQUEST_V0;
 
     private RenewDelegationTokenRequest(short version, ByteBuffer hmac, long renewTimePeriod) {
-        super(version);
+        super(ApiKeys.RENEW_DELEGATION_TOKEN, version);
 
         this.hmac = hmac;
         this.renewTimePeriod = renewTimePeriod;
     }
 
     public RenewDelegationTokenRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.RENEW_DELEGATION_TOKEN, versionId);
 
         hmac = struct.getBytes(HMAC_KEY_NAME);
         renewTimePeriod = struct.getLong(RENEW_TIME_PERIOD_KEY_NAME);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
index 7638c6c..24c2fbe 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
@@ -20,12 +20,16 @@ import org.apache.kafka.common.acl.AccessControlEntry;
 import org.apache.kafka.common.acl.AccessControlEntryFilter;
 import org.apache.kafka.common.acl.AclOperation;
 import org.apache.kafka.common.acl.AclPermissionType;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.record.RecordBatch;
 import org.apache.kafka.common.resource.PatternType;
 import org.apache.kafka.common.resource.ResourcePattern;
 import org.apache.kafka.common.resource.ResourcePatternFilter;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.resource.ResourceType;
 
+import java.util.Optional;
+
 import static org.apache.kafka.common.protocol.CommonFields.HOST;
 import static org.apache.kafka.common.protocol.CommonFields.HOST_FILTER;
 import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
@@ -101,4 +105,16 @@ final class RequestUtils {
         struct.set(OPERATION, filter.operation().code());
         struct.set(PERMISSION_TYPE, filter.permissionType().code());
     }
+
+    static void setLeaderEpochIfExists(Struct struct, Field.Int32 leaderEpochField, Optional<Integer> leaderEpoch) {
+        struct.setIfExists(leaderEpochField, leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH));
+    }
+
+    static Optional<Integer> getLeaderEpoch(Struct struct, Field.Int32 leaderEpochField) {
+        int leaderEpoch = struct.getOrElse(leaderEpochField, RecordBatch.NO_PARTITION_LEADER_EPOCH);
+        Optional<Integer> leaderEpochOpt = leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH ?
+                Optional.empty() : Optional.of(leaderEpoch);
+        return leaderEpochOpt;
+    }
+
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
index 74d31a6..2ce144e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
@@ -72,12 +72,12 @@ public class SaslAuthenticateRequest extends AbstractRequest {
     }
 
     public SaslAuthenticateRequest(ByteBuffer saslAuthBytes, short version) {
-        super(version);
+        super(ApiKeys.SASL_AUTHENTICATE, version);
         this.saslAuthBytes = saslAuthBytes;
     }
 
     public SaslAuthenticateRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.SASL_AUTHENTICATE, version);
         saslAuthBytes = struct.getBytes(SASL_AUTH_BYTES_KEY_NAME);
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
index a06a4db..7225eb7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
@@ -80,12 +80,12 @@ public class SaslHandshakeRequest extends AbstractRequest {
     }
 
     public SaslHandshakeRequest(String mechanism, short version) {
-        super(version);
+        super(ApiKeys.SASL_HANDSHAKE, version);
         this.mechanism = mechanism;
     }
 
     public SaslHandshakeRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.SASL_HANDSHAKE, version);
         mechanism = struct.getString(MECHANISM_KEY_NAME);
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
index d79c938..a296c80 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
@@ -98,7 +98,7 @@ public class StopReplicaRequest extends AbstractRequest {
 
     private StopReplicaRequest(int controllerId, int controllerEpoch, boolean deletePartitions,
                                Set<TopicPartition> partitions, short version) {
-        super(version);
+        super(ApiKeys.STOP_REPLICA, version);
         this.controllerId = controllerId;
         this.controllerEpoch = controllerEpoch;
         this.deletePartitions = deletePartitions;
@@ -106,7 +106,7 @@ public class StopReplicaRequest extends AbstractRequest {
     }
 
     public StopReplicaRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.STOP_REPLICA, version);
 
         partitions = new HashSet<>();
         for (Object partitionDataObj : struct.getArray(PARTITIONS_KEY_NAME)) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
index 962bc77..237320f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
@@ -101,7 +101,7 @@ public class SyncGroupRequest extends AbstractRequest {
 
     private SyncGroupRequest(String groupId, int generationId, String memberId,
                              Map<String, ByteBuffer> groupAssignment, short version) {
-        super(version);
+        super(ApiKeys.SYNC_GROUP, version);
         this.groupId = groupId;
         this.generationId = generationId;
         this.memberId = memberId;
@@ -109,7 +109,7 @@ public class SyncGroupRequest extends AbstractRequest {
     }
 
     public SyncGroupRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.SYNC_GROUP, version);
         this.groupId = struct.get(GROUP_ID);
         this.generationId = struct.get(GENERATION_ID);
         this.memberId = struct.get(MEMBER_ID);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
index 25245be..1c922e1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -28,44 +27,66 @@ import org.apache.kafka.common.utils.CollectionUtils;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Optional;
 
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_LEADER_EPOCH;
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_METADATA;
+import static org.apache.kafka.common.protocol.CommonFields.COMMITTED_OFFSET;
 import static org.apache.kafka.common.protocol.CommonFields.GROUP_ID;
 import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.PRODUCER_EPOCH;
 import static org.apache.kafka.common.protocol.CommonFields.PRODUCER_ID;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 import static org.apache.kafka.common.protocol.CommonFields.TRANSACTIONAL_ID;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
-import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
 
 public class TxnOffsetCommitRequest extends AbstractRequest {
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String OFFSET_KEY_NAME = "offset";
-    private static final String METADATA_KEY_NAME = "metadata";
+    // top level fields
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "Topics to commit offsets");
 
-    private static final Schema TXN_OFFSET_COMMIT_PARTITION_OFFSET_METADATA_REQUEST_V0 = new Schema(
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "Partitions to commit offsets");
+
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
-            new Field(OFFSET_KEY_NAME, INT64),
-            new Field(METADATA_KEY_NAME, NULLABLE_STRING));
+            COMMITTED_OFFSET,
+            COMMITTED_METADATA);
+
+    private static final Field TOPICS_V0 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V0);
 
     private static final Schema TXN_OFFSET_COMMIT_REQUEST_V0 = new Schema(
             TRANSACTIONAL_ID,
             GROUP_ID,
             PRODUCER_ID,
             PRODUCER_EPOCH,
-            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
-                    TOPIC_NAME,
-                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(TXN_OFFSET_COMMIT_PARTITION_OFFSET_METADATA_REQUEST_V0)))),
-                    "The partitions to write markers for."));
-
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+            TOPICS_V0);
+
+    // V1 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema TXN_OFFSET_COMMIT_REQUEST_V1 = TXN_OFFSET_COMMIT_REQUEST_V0;
 
+    // V2 adds the leader epoch to the partition data
+    private static final Field PARTITIONS_V2 = PARTITIONS.withFields(
+            PARTITION_ID,
+            COMMITTED_OFFSET,
+            COMMITTED_LEADER_EPOCH,
+            COMMITTED_METADATA);
+
+    private static final Field TOPICS_V2 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V2);
+
+    private static final Schema TXN_OFFSET_COMMIT_REQUEST_V2 = new Schema(
+            TRANSACTIONAL_ID,
+            GROUP_ID,
+            PRODUCER_ID,
+            PRODUCER_EPOCH,
+            TOPICS_V2);
+
     public static Schema[] schemaVersions() {
-        return new Schema[]{TXN_OFFSET_COMMIT_REQUEST_V0, TXN_OFFSET_COMMIT_REQUEST_V1};
+        return new Schema[]{TXN_OFFSET_COMMIT_REQUEST_V0, TXN_OFFSET_COMMIT_REQUEST_V1, TXN_OFFSET_COMMIT_REQUEST_V2};
     }
 
     public static class Builder extends AbstractRequest.Builder<TxnOffsetCommitRequest> {
@@ -120,7 +141,7 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
 
     public TxnOffsetCommitRequest(short version, String transactionalId, String consumerGroupId, long producerId,
                                   short producerEpoch, Map<TopicPartition, CommittedOffset> offsets) {
-        super(version);
+        super(ApiKeys.TXN_OFFSET_COMMIT, version);
         this.transactionalId = transactionalId;
         this.consumerGroupId = consumerGroupId;
         this.producerId = producerId;
@@ -129,23 +150,24 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
     }
 
     public TxnOffsetCommitRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.TXN_OFFSET_COMMIT, version);
         this.transactionalId = struct.get(TRANSACTIONAL_ID);
         this.consumerGroupId = struct.get(GROUP_ID);
         this.producerId = struct.get(PRODUCER_ID);
         this.producerEpoch = struct.get(PRODUCER_EPOCH);
 
         Map<TopicPartition, CommittedOffset> offsets = new HashMap<>();
-        Object[] topicPartitionsArray = struct.getArray(TOPICS_KEY_NAME);
+        Object[] topicPartitionsArray = struct.get(TOPICS);
         for (Object topicPartitionObj : topicPartitionsArray) {
             Struct topicPartitionStruct = (Struct) topicPartitionObj;
             String topic = topicPartitionStruct.get(TOPIC_NAME);
-            for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionObj : topicPartitionStruct.get(PARTITIONS)) {
                 Struct partitionStruct = (Struct) partitionObj;
                 TopicPartition partition = new TopicPartition(topic, partitionStruct.get(PARTITION_ID));
-                long offset = partitionStruct.getLong(OFFSET_KEY_NAME);
-                String metadata = partitionStruct.getString(METADATA_KEY_NAME);
-                offsets.put(partition, new CommittedOffset(offset, metadata));
+                long offset = partitionStruct.get(COMMITTED_OFFSET);
+                String metadata = partitionStruct.get(COMMITTED_METADATA);
+                Optional<Integer> leaderEpoch = RequestUtils.getLeaderEpoch(partitionStruct, COMMITTED_LEADER_EPOCH);
+                offsets.put(partition, new CommittedOffset(offset, metadata, leaderEpoch));
             }
         }
         this.offsets = offsets;
@@ -179,29 +201,31 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
         struct.set(PRODUCER_ID, producerId);
         struct.set(PRODUCER_EPOCH, producerEpoch);
 
-        Map<String, Map<Integer, CommittedOffset>> mappedPartitionOffsets = CollectionUtils.groupDataByTopic(offsets);
+        Map<String, Map<Integer, CommittedOffset>> mappedPartitionOffsets = CollectionUtils.groupPartitionDataByTopic(offsets);
         Object[] partitionsArray = new Object[mappedPartitionOffsets.size()];
         int i = 0;
         for (Map.Entry<String, Map<Integer, CommittedOffset>> topicAndPartitions : mappedPartitionOffsets.entrySet()) {
-            Struct topicPartitionsStruct = struct.instance(TOPICS_KEY_NAME);
+            Struct topicPartitionsStruct = struct.instance(TOPICS);
             topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
 
             Map<Integer, CommittedOffset> partitionOffsets = topicAndPartitions.getValue();
             Object[] partitionOffsetsArray = new Object[partitionOffsets.size()];
             int j = 0;
             for (Map.Entry<Integer, CommittedOffset> partitionOffset : partitionOffsets.entrySet()) {
-                Struct partitionOffsetStruct = topicPartitionsStruct.instance(PARTITIONS_KEY_NAME);
+                Struct partitionOffsetStruct = topicPartitionsStruct.instance(PARTITIONS);
                 partitionOffsetStruct.set(PARTITION_ID, partitionOffset.getKey());
                 CommittedOffset committedOffset = partitionOffset.getValue();
-                partitionOffsetStruct.set(OFFSET_KEY_NAME, committedOffset.offset);
-                partitionOffsetStruct.set(METADATA_KEY_NAME, committedOffset.metadata);
+                partitionOffsetStruct.set(COMMITTED_OFFSET, committedOffset.offset);
+                partitionOffsetStruct.set(COMMITTED_METADATA, committedOffset.metadata);
+                RequestUtils.setLeaderEpochIfExists(partitionOffsetStruct, COMMITTED_LEADER_EPOCH,
+                        committedOffset.leaderEpoch);
                 partitionOffsetsArray[j++] = partitionOffsetStruct;
             }
-            topicPartitionsStruct.set(PARTITIONS_KEY_NAME, partitionOffsetsArray);
+            topicPartitionsStruct.set(PARTITIONS, partitionOffsetsArray);
             partitionsArray[i++] = topicPartitionsStruct;
         }
 
-        struct.set(TOPICS_KEY_NAME, partitionsArray);
+        struct.set(TOPICS, partitionsArray);
         return struct;
     }
 
@@ -219,28 +243,23 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
     }
 
     public static class CommittedOffset {
-        private final long offset;
-        private final String metadata;
+        public final long offset;
+        public final String metadata;
+        public final Optional<Integer> leaderEpoch;
 
-        public CommittedOffset(long offset, String metadata) {
+        public CommittedOffset(long offset, String metadata, Optional<Integer> leaderEpoch) {
             this.offset = offset;
             this.metadata = metadata;
+            this.leaderEpoch = leaderEpoch;
         }
 
         @Override
         public String toString() {
             return "CommittedOffset(" +
                     "offset=" + offset +
+                    ", leaderEpoch=" + leaderEpoch +
                     ", metadata='" + metadata + "')";
         }
-
-        public long offset() {
-            return offset;
-        }
-
-        public String metadata() {
-            return metadata;
-        }
     }
 
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
index c34fd40..ba8f7d6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
@@ -19,7 +19,6 @@ 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.Errors;
-import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
@@ -34,41 +33,49 @@ import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
 import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
 import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 
+/**
+ *
+ * Possible error codes:
+ *   InvalidProducerEpoch
+ *   NotCoordinator
+ *   CoordinatorNotAvailable
+ *   CoordinatorLoadInProgress
+ *   OffsetMetadataTooLarge
+ *   GroupAuthorizationFailed
+ *   InvalidCommitOffsetSize
+ *   TransactionalIdAuthorizationFailed
+ *   RequestTimedOut
+ */
 public class TxnOffsetCommitResponse extends AbstractResponse {
-    private static final String TOPICS_KEY_NAME = "topics";
-    private static final String PARTITIONS_KEY_NAME = "partitions";
+    private static final Field.ComplexArray TOPICS = new Field.ComplexArray("topics",
+            "Responses by topic for committed offsets");
 
-    private static final Schema TXN_OFFSET_COMMIT_PARTITION_ERROR_RESPONSE_V0 = new Schema(
+    // topic level fields
+    private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions",
+            "Responses by partition for committed offsets");
+
+    private static final Field PARTITIONS_V0 = PARTITIONS.withFields(
             PARTITION_ID,
             ERROR_CODE);
 
+    private static final Field TOPICS_V0 = TOPICS.withFields(
+            TOPIC_NAME,
+            PARTITIONS_V0);
+
     private static final Schema TXN_OFFSET_COMMIT_RESPONSE_V0 = new Schema(
             THROTTLE_TIME_MS,
-            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
-                    TOPIC_NAME,
-                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(TXN_OFFSET_COMMIT_PARTITION_ERROR_RESPONSE_V0)))),
-                    "Errors per partition from writing markers."));
-
-    /**
-     * The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
-     */
+            TOPICS_V0);
+
+    // V1 bump used to indicate that on quota violation brokers send out responses before throttling.
     private static final Schema TXN_OFFSET_COMMIT_RESPONSE_V1 = TXN_OFFSET_COMMIT_RESPONSE_V0;
 
+    // V2 adds the leader epoch to the partition data
+    private static final Schema TXN_OFFSET_COMMIT_RESPONSE_V2 = TXN_OFFSET_COMMIT_RESPONSE_V1;
+
     public static Schema[] schemaVersions() {
-        return new Schema[]{TXN_OFFSET_COMMIT_RESPONSE_V0, TXN_OFFSET_COMMIT_RESPONSE_V1};
+        return new Schema[]{TXN_OFFSET_COMMIT_RESPONSE_V0, TXN_OFFSET_COMMIT_RESPONSE_V1, TXN_OFFSET_COMMIT_RESPONSE_V2};
     }
 
-    // Possible error codes:
-    //   InvalidProducerEpoch
-    //   NotCoordinator
-    //   CoordinatorNotAvailable
-    //   CoordinatorLoadInProgress
-    //   OffsetMetadataTooLarge
-    //   GroupAuthorizationFailed
-    //   InvalidCommitOffsetSize
-    //   TransactionalIdAuthorizationFailed
-    //   RequestTimedOut
-
     private final Map<TopicPartition, Errors> errors;
     private final int throttleTimeMs;
 
@@ -80,11 +87,11 @@ public class TxnOffsetCommitResponse extends AbstractResponse {
     public TxnOffsetCommitResponse(Struct struct) {
         this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         Map<TopicPartition, Errors> errors = new HashMap<>();
-        Object[] topicPartitionsArray = struct.getArray(TOPICS_KEY_NAME);
+        Object[] topicPartitionsArray = struct.get(TOPICS);
         for (Object topicPartitionObj : topicPartitionsArray) {
             Struct topicPartitionStruct = (Struct) topicPartitionObj;
             String topic = topicPartitionStruct.get(TOPIC_NAME);
-            for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
+            for (Object partitionObj : topicPartitionStruct.get(PARTITIONS)) {
                 Struct partitionStruct = (Struct) partitionObj;
                 Integer partition = partitionStruct.get(PARTITION_ID);
                 Errors error = Errors.forCode(partitionStruct.get(ERROR_CODE));
@@ -98,27 +105,27 @@ public class TxnOffsetCommitResponse extends AbstractResponse {
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.TXN_OFFSET_COMMIT.responseSchema(version));
         struct.set(THROTTLE_TIME_MS, throttleTimeMs);
-        Map<String, Map<Integer, Errors>> mappedPartitions = CollectionUtils.groupDataByTopic(errors);
+        Map<String, Map<Integer, Errors>> mappedPartitions = CollectionUtils.groupPartitionDataByTopic(errors);
         Object[] partitionsArray = new Object[mappedPartitions.size()];
         int i = 0;
         for (Map.Entry<String, Map<Integer, Errors>> topicAndPartitions : mappedPartitions.entrySet()) {
-            Struct topicPartitionsStruct = struct.instance(TOPICS_KEY_NAME);
+            Struct topicPartitionsStruct = struct.instance(TOPICS);
             topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
             Map<Integer, Errors> partitionAndErrors = topicAndPartitions.getValue();
 
             Object[] partitionAndErrorsArray = new Object[partitionAndErrors.size()];
             int j = 0;
             for (Map.Entry<Integer, Errors> partitionAndError : partitionAndErrors.entrySet()) {
-                Struct partitionAndErrorStruct = topicPartitionsStruct.instance(PARTITIONS_KEY_NAME);
+                Struct partitionAndErrorStruct = topicPartitionsStruct.instance(PARTITIONS);
                 partitionAndErrorStruct.set(PARTITION_ID, partitionAndError.getKey());
                 partitionAndErrorStruct.set(ERROR_CODE, partitionAndError.getValue().code());
                 partitionAndErrorsArray[j++] = partitionAndErrorStruct;
             }
-            topicPartitionsStruct.set(PARTITIONS_KEY_NAME, partitionAndErrorsArray);
+            topicPartitionsStruct.set(PARTITIONS, partitionAndErrorsArray);
             partitionsArray[i++] = topicPartitionsStruct;
         }
 
-        struct.set(TOPICS_KEY_NAME, partitionsArray);
+        struct.set(TOPICS, partitionsArray);
         return struct;
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
index a273765..5c7b6f7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
@@ -284,7 +284,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
 
     private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch,
                                   Map<TopicPartition, PartitionState> partitionStates, Set<Broker> liveBrokers) {
-        super(version);
+        super(ApiKeys.UPDATE_METADATA, version);
         this.controllerId = controllerId;
         this.controllerEpoch = controllerEpoch;
         this.partitionStates = partitionStates;
@@ -292,7 +292,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
     }
 
     public UpdateMetadataRequest(Struct struct, short versionId) {
-        super(versionId);
+        super(ApiKeys.UPDATE_METADATA, versionId);
         Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
         for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) {
             Struct partitionStateData = (Struct) partitionStateDataObj;
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
index 3f7a0c9..33f9bb5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
@@ -153,13 +153,13 @@ public class WriteTxnMarkersRequest extends AbstractRequest {
     private final List<TxnMarkerEntry> markers;
 
     private WriteTxnMarkersRequest(short version, List<TxnMarkerEntry> markers) {
-        super(version);
+        super(ApiKeys.WRITE_TXN_MARKERS, version);
 
         this.markers = markers;
     }
 
     public WriteTxnMarkersRequest(Struct struct, short version) {
-        super(version);
+        super(ApiKeys.WRITE_TXN_MARKERS, version);
         List<TxnMarkerEntry> markers = new ArrayList<>();
         Object[] markersArray = struct.getArray(TXN_MARKERS_KEY_NAME);
         for (Object markerObj : markersArray) {
@@ -204,7 +204,7 @@ public class WriteTxnMarkersRequest extends AbstractRequest {
             markerStruct.set(COORDINATOR_EPOCH_KEY_NAME, entry.coordinatorEpoch);
             markerStruct.set(TRANSACTION_RESULT_KEY_NAME, entry.result.id);
 
-            Map<String, List<Integer>> mappedPartitions = CollectionUtils.groupDataByTopic(entry.partitions);
+            Map<String, List<Integer>> mappedPartitions = CollectionUtils.groupPartitionsByTopic(entry.partitions);
             Object[] partitionsArray = new Object[mappedPartitions.size()];
             int j = 0;
             for (Map.Entry<String, List<Integer>> topicAndPartitions : mappedPartitions.entrySet()) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
index f307760..92b5fc0 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
@@ -118,7 +118,7 @@ public class WriteTxnMarkersResponse extends AbstractResponse {
             responseStruct.set(PRODUCER_ID_KEY_NAME, responseEntry.getKey());
 
             Map<TopicPartition, Errors> partitionAndErrors = responseEntry.getValue();
-            Map<String, Map<Integer, Errors>> mappedPartitions = CollectionUtils.groupDataByTopic(partitionAndErrors);
+            Map<String, Map<Integer, Errors>> mappedPartitions = CollectionUtils.groupPartitionDataByTopic(partitionAndErrors);
             Object[] partitionsArray = new Object[mappedPartitions.size()];
             int i = 0;
             for (Map.Entry<String, Map<Integer, Errors>> topicAndPartitions : mappedPartitions.entrySet()) {
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
index 04fce64..3489728 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
@@ -39,41 +39,36 @@ public final class CollectionUtils {
 
     /**
      * group data by topic
+     *
      * @param data Data to be partitioned
      * @param <T> Partition data type
      * @return partitioned data
      */
-    public static <T> Map<String, Map<Integer, T>> groupDataByTopic(Map<TopicPartition, ? extends T> data) {
+    public static <T> Map<String, Map<Integer, T>> groupPartitionDataByTopic(Map<TopicPartition, ? extends T> data) {
         Map<String, Map<Integer, T>> dataByTopic = new HashMap<>();
-        for (Map.Entry<TopicPartition, ? extends T> entry: data.entrySet()) {
+        for (Map.Entry<TopicPartition, ? extends T> entry : data.entrySet()) {
             String topic = entry.getKey().topic();
             int partition = entry.getKey().partition();
-            Map<Integer, T> topicData = dataByTopic.get(topic);
-            if (topicData == null) {
-                topicData = new HashMap<>();
-                dataByTopic.put(topic, topicData);
-            }
+            Map<Integer, T> topicData = dataByTopic.computeIfAbsent(topic, t -> new HashMap<>());
             topicData.put(partition, entry.getValue());
         }
         return dataByTopic;
     }
 
     /**
-     * group partitions by topic
-     * @param partitions
+     * Group a list of partitions by the topic name.
+     *
+     * @param partitions The partitions to collect
      * @return partitions per topic
      */
-    public static Map<String, List<Integer>> groupDataByTopic(List<TopicPartition> partitions) {
+    public static Map<String, List<Integer>> groupPartitionsByTopic(List<TopicPartition> partitions) {
         Map<String, List<Integer>> partitionsByTopic = new HashMap<>();
-        for (TopicPartition tp: partitions) {
+        for (TopicPartition tp : partitions) {
             String topic = tp.topic();
-            List<Integer> topicData = partitionsByTopic.get(topic);
-            if (topicData == null) {
-                topicData = new ArrayList<>();
-                partitionsByTopic.put(topic, topicData);
-            }
+            List<Integer> topicData = partitionsByTopic.computeIfAbsent(topic, t -> new ArrayList<>());
             topicData.add(tp.partition());
         }
-        return  partitionsByTopic;
+        return partitionsByTopic;
     }
+
 }
diff --git a/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
index 3095717..4c12fc6 100644
--- a/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
@@ -18,13 +18,13 @@ package org.apache.kafka.clients;
 
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.utils.LogContext;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
-import org.slf4j.Logger;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -33,6 +33,7 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -52,13 +53,11 @@ public class FetchSessionHandlerTest {
 
     private static final LogContext LOG_CONTEXT = new LogContext("[FetchSessionHandler]=");
 
-    private static final Logger log = LOG_CONTEXT.logger(FetchSessionHandler.class);
-
     /**
      * Create a set of TopicPartitions.  We use a TreeSet, in order to get a deterministic
      * ordering for test purposes.
      */
-    private final static Set<TopicPartition> toSet(TopicPartition... arr) {
+    private static Set<TopicPartition> toSet(TopicPartition... arr) {
         TreeSet<TopicPartition> set = new TreeSet<>(new Comparator<TopicPartition>() {
             @Override
             public int compare(TopicPartition o1, TopicPartition o2) {
@@ -70,7 +69,7 @@ public class FetchSessionHandlerTest {
     }
 
     @Test
-    public void testFindMissing() throws Exception {
+    public void testFindMissing() {
         TopicPartition foo0 = new TopicPartition("foo", 0);
         TopicPartition foo1 = new TopicPartition("foo", 1);
         TopicPartition bar0 = new TopicPartition("bar", 0);
@@ -95,7 +94,7 @@ public class FetchSessionHandlerTest {
 
         ReqEntry(String topic, int partition, long fetchOffset, long logStartOffset, int maxBytes) {
             this.part = new TopicPartition(topic, partition);
-            this.data = new FetchRequest.PartitionData(fetchOffset, logStartOffset, maxBytes);
+            this.data = new FetchRequest.PartitionData(fetchOffset, logStartOffset, maxBytes, Optional.empty());
         }
     }
 
@@ -153,11 +152,11 @@ public class FetchSessionHandlerTest {
 
     private static final class RespEntry {
         final TopicPartition part;
-        final FetchResponse.PartitionData data;
+        final FetchResponse.PartitionData<MemoryRecords> data;
 
         RespEntry(String topic, int partition, long highWatermark, long lastStableOffset) {
             this.part = new TopicPartition(topic, partition);
-            this.data = new FetchResponse.PartitionData(
+            this.data = new FetchResponse.PartitionData<>(
                 Errors.NONE,
                 highWatermark,
                 lastStableOffset,
@@ -167,8 +166,8 @@ public class FetchSessionHandlerTest {
         }
     }
 
-    private static LinkedHashMap<TopicPartition, FetchResponse.PartitionData> respMap(RespEntry... entries) {
-        LinkedHashMap<TopicPartition, FetchResponse.PartitionData> map = new LinkedHashMap<>();
+    private static LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> respMap(RespEntry... entries) {
+        LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> map = new LinkedHashMap<>();
         for (RespEntry entry : entries) {
             map.put(entry.part, entry.data);
         }
@@ -180,13 +179,13 @@ public class FetchSessionHandlerTest {
      * Pre-KIP-227 brokers always supply this kind of response.
      */
     @Test
-    public void testSessionless() throws Exception {
+    public void testSessionless() {
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
         FetchSessionHandler.Builder builder = handler.newBuilder();
         builder.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         builder.add(new TopicPartition("foo", 1),
-            new FetchRequest.PartitionData(10, 110, 210));
+            new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
         assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
                                new ReqEntry("foo", 1, 10, 110, 210)),
@@ -194,7 +193,7 @@ public class FetchSessionHandlerTest {
         assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
         assertEquals(INITIAL_EPOCH, data.metadata().epoch());
 
-        FetchResponse resp = new FetchResponse(Errors.NONE,
+        FetchResponse<MemoryRecords> resp = new FetchResponse<>(Errors.NONE,
             respMap(new RespEntry("foo", 0, 0, 0),
                     new RespEntry("foo", 1, 0, 0)),
             0, INVALID_SESSION_ID);
@@ -202,7 +201,7 @@ public class FetchSessionHandlerTest {
 
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         builder2.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
         assertEquals(INVALID_SESSION_ID, data2.metadata().sessionId());
         assertEquals(INITIAL_EPOCH, data2.metadata().epoch());
@@ -214,13 +213,13 @@ public class FetchSessionHandlerTest {
      * Test handling an incremental fetch session.
      */
     @Test
-    public void testIncrementals() throws Exception {
+    public void testIncrementals() {
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
         FetchSessionHandler.Builder builder = handler.newBuilder();
         builder.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         builder.add(new TopicPartition("foo", 1),
-            new FetchRequest.PartitionData(10, 110, 210));
+            new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
         assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
             new ReqEntry("foo", 1, 10, 110, 210)),
@@ -228,7 +227,7 @@ public class FetchSessionHandlerTest {
         assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
         assertEquals(INITIAL_EPOCH, data.metadata().epoch());
 
-        FetchResponse resp = new FetchResponse(Errors.NONE,
+        FetchResponse<MemoryRecords> resp = new FetchResponse<>(Errors.NONE,
             respMap(new RespEntry("foo", 0, 10, 20),
                     new RespEntry("foo", 1, 10, 20)),
             0, 123);
@@ -237,11 +236,11 @@ public class FetchSessionHandlerTest {
         // Test an incremental fetch request which adds one partition and modifies another.
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         builder2.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         builder2.add(new TopicPartition("foo", 1),
-            new FetchRequest.PartitionData(10, 120, 210));
+            new FetchRequest.PartitionData(10, 120, 210, Optional.empty()));
         builder2.add(new TopicPartition("bar", 0),
-            new FetchRequest.PartitionData(20, 200, 200));
+            new FetchRequest.PartitionData(20, 200, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
         assertFalse(data2.metadata().isFull());
         assertMapEquals(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
@@ -252,24 +251,24 @@ public class FetchSessionHandlerTest {
                 new ReqEntry("foo", 1, 10, 120, 210)),
             data2.toSend());
 
-        FetchResponse resp2 = new FetchResponse(Errors.NONE,
+        FetchResponse<MemoryRecords> resp2 = new FetchResponse<>(Errors.NONE,
             respMap(new RespEntry("foo", 1, 20, 20)),
             0, 123);
         handler.handleResponse(resp2);
 
         // Skip building a new request.  Test that handling an invalid fetch session epoch response results
         // in a request which closes the session.
-        FetchResponse resp3 = new FetchResponse(Errors.INVALID_FETCH_SESSION_EPOCH, respMap(),
+        FetchResponse<MemoryRecords> resp3 = new FetchResponse<>(Errors.INVALID_FETCH_SESSION_EPOCH, respMap(),
             0, INVALID_SESSION_ID);
         handler.handleResponse(resp3);
 
         FetchSessionHandler.Builder builder4 = handler.newBuilder();
         builder4.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         builder4.add(new TopicPartition("foo", 1),
-            new FetchRequest.PartitionData(10, 120, 210));
+            new FetchRequest.PartitionData(10, 120, 210, Optional.empty()));
         builder4.add(new TopicPartition("bar", 0),
-            new FetchRequest.PartitionData(20, 200, 200));
+            new FetchRequest.PartitionData(20, 200, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data4 = builder4.build();
         assertTrue(data4.metadata().isFull());
         assertEquals(data2.metadata().sessionId(), data4.metadata().sessionId());
@@ -284,11 +283,11 @@ public class FetchSessionHandlerTest {
      * Test that calling FetchSessionHandler#Builder#build twice fails.
      */
     @Test
-    public void testDoubleBuild() throws Exception {
+    public void testDoubleBuild() {
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
         FetchSessionHandler.Builder builder = handler.newBuilder();
         builder.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         builder.build();
         try {
             builder.build();
@@ -299,15 +298,15 @@ public class FetchSessionHandlerTest {
     }
 
     @Test
-    public void testIncrementalPartitionRemoval() throws Exception {
+    public void testIncrementalPartitionRemoval() {
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
         FetchSessionHandler.Builder builder = handler.newBuilder();
         builder.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         builder.add(new TopicPartition("foo", 1),
-            new FetchRequest.PartitionData(10, 110, 210));
+            new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
         builder.add(new TopicPartition("bar", 0),
-            new FetchRequest.PartitionData(20, 120, 220));
+            new FetchRequest.PartitionData(20, 120, 220, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
         assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
             new ReqEntry("foo", 1, 10, 110, 210),
@@ -315,7 +314,7 @@ public class FetchSessionHandlerTest {
             data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
 
-        FetchResponse resp = new FetchResponse(Errors.NONE,
+        FetchResponse<MemoryRecords> resp = new FetchResponse<>(Errors.NONE,
             respMap(new RespEntry("foo", 0, 10, 20),
                     new RespEntry("foo", 1, 10, 20),
                     new RespEntry("bar", 0, 10, 20)),
@@ -325,7 +324,7 @@ public class FetchSessionHandlerTest {
         // Test an incremental fetch request which removes two partitions.
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         builder2.add(new TopicPartition("foo", 1),
-            new FetchRequest.PartitionData(10, 110, 210));
+            new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
         assertFalse(data2.metadata().isFull());
         assertEquals(123, data2.metadata().sessionId());
@@ -340,12 +339,12 @@ public class FetchSessionHandlerTest {
 
         // A FETCH_SESSION_ID_NOT_FOUND response triggers us to close the session.
         // The next request is a session establishing FULL request.
-        FetchResponse resp2 = new FetchResponse(Errors.FETCH_SESSION_ID_NOT_FOUND,
+        FetchResponse<MemoryRecords> resp2 = new FetchResponse<>(Errors.FETCH_SESSION_ID_NOT_FOUND,
             respMap(), 0, INVALID_SESSION_ID);
         handler.handleResponse(resp2);
         FetchSessionHandler.Builder builder3 = handler.newBuilder();
         builder3.add(new TopicPartition("foo", 0),
-            new FetchRequest.PartitionData(0, 100, 200));
+            new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data3 = builder3.build();
         assertTrue(data3.metadata().isFull());
         assertEquals(INVALID_SESSION_ID, data3.metadata().sessionId());
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
index c0dc542..34432c3 100644
--- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
@@ -94,6 +94,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -462,12 +463,13 @@ public class KafkaAdminClientTest {
             env.kafkaClient().prepareResponse(new MetadataResponse(initializedCluster.nodes(),
                     initializedCluster.clusterResource().clusterId(),
                     initializedCluster.controller().id(),
-                    Collections.<MetadataResponse.TopicMetadata>emptyList()));
+                    Collections.emptyList()));
 
             // Then we respond to the DescribeTopic request
             Node leader = initializedCluster.nodes().get(0);
             MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(
-                    Errors.NONE, 0, leader, singletonList(leader), singletonList(leader), singletonList(leader));
+                    Errors.NONE, 0, leader, Optional.of(10), singletonList(leader),
+                    singletonList(leader), singletonList(leader));
             env.kafkaClient().prepareResponse(new MetadataResponse(initializedCluster.nodes(),
                     initializedCluster.clusterResource().clusterId(), 1,
                     singletonList(new MetadataResponse.TopicMetadata(Errors.NONE, topic, false,
@@ -788,16 +790,17 @@ public class KafkaAdminClientTest {
 
             List<MetadataResponse.TopicMetadata> t = new ArrayList<>();
             List<MetadataResponse.PartitionMetadata> p = new ArrayList<>();
-            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 0, nodes.get(0),
-                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.<Node>emptyList()));
-            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, nodes.get(0),
-                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.<Node>emptyList()));
+            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 0, nodes.get(0), Optional.of(5),
+                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.emptyList()));
+            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, nodes.get(0), Optional.of(5),
+                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.emptyList()));
             p.add(new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, 2, null,
-                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.<Node>emptyList()));
-            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 3, nodes.get(0),
-                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.<Node>emptyList()));
-            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 4, nodes.get(0),
-                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.<Node>emptyList()));
+                    Optional.empty(), singletonList(nodes.get(0)), singletonList(nodes.get(0)),
+                    Collections.emptyList()));
+            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 3, nodes.get(0), Optional.of(5),
+                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.emptyList()));
+            p.add(new MetadataResponse.PartitionMetadata(Errors.NONE, 4, nodes.get(0), Optional.of(5),
+                    singletonList(nodes.get(0)), singletonList(nodes.get(0)), Collections.emptyList()));
 
             t.add(new MetadataResponse.TopicMetadata(Errors.NONE, "my_topic", false, p));
 
@@ -1068,9 +1071,9 @@ public class KafkaAdminClientTest {
             new Cluster(
                 "mockClusterId",
                 nodes.values(),
-                Collections.<PartitionInfo>emptyList(),
-                Collections.<String>emptySet(),
-                Collections.<String>emptySet(), nodes.get(0));
+                Collections.emptyList(),
+                Collections.emptySet(),
+                Collections.emptySet(), nodes.get(0));
 
         try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) {
             env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
@@ -1083,9 +1086,12 @@ public class KafkaAdminClientTest {
             TopicPartition myTopicPartition2 = new TopicPartition("my_topic", 2);
 
             final Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<>();
-            responseData.put(myTopicPartition0, new OffsetFetchResponse.PartitionData(10, "", Errors.NONE));
-            responseData.put(myTopicPartition1, new OffsetFetchResponse.PartitionData(0, "", Errors.NONE));
-            responseData.put(myTopicPartition2, new OffsetFetchResponse.PartitionData(20, "", Errors.NONE));
+            responseData.put(myTopicPartition0, new OffsetFetchResponse.PartitionData(10,
+                    Optional.empty(), "", Errors.NONE));
+            responseData.put(myTopicPartition1, new OffsetFetchResponse.PartitionData(0,
+                    Optional.empty(), "", Errors.NONE));
+            responseData.put(myTopicPartition2, new OffsetFetchResponse.PartitionData(20,
+                    Optional.empty(), "", Errors.NONE));
             env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NONE, responseData));
 
             final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets("group-0");
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java
index 3273645..aae269a 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java
@@ -21,6 +21,8 @@ import org.apache.kafka.common.record.DefaultRecord;
 import org.apache.kafka.common.record.TimestampType;
 import org.junit.Test;
 
+import java.util.Optional;
+
 import static org.junit.Assert.assertEquals;
 
 public class ConsumerRecordTest {
@@ -45,6 +47,7 @@ public class ConsumerRecordTest {
         assertEquals(ConsumerRecord.NULL_CHECKSUM, record.checksum());
         assertEquals(ConsumerRecord.NULL_SIZE, record.serializedKeySize());
         assertEquals(ConsumerRecord.NULL_SIZE, record.serializedValueSize());
+        assertEquals(Optional.empty(), record.leaderEpoch());
         assertEquals(new RecordHeaders(), record.headers());
     }
 
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index a0f95c4..8b8cea6 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.clients.consumer;
 
-import java.util.ArrayList;
 import org.apache.kafka.clients.ClientRequest;
 import org.apache.kafka.clients.KafkaClient;
 import org.apache.kafka.clients.Metadata;
@@ -83,6 +82,7 @@ import org.junit.rules.ExpectedException;
 
 import java.nio.ByteBuffer;
 import java.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -91,6 +91,7 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
 import java.util.Queue;
 import java.util.Set;
@@ -499,7 +500,7 @@ public class KafkaConsumerTest {
         Node node = cluster.nodes().get(0);
 
         Metadata metadata = createMetadata();
-        metadata.update(cluster, Collections.<String>emptySet(), time.milliseconds());
+        metadata.update(cluster, Collections.emptySet(), time.milliseconds());
 
         MockClient client = new MockClient(time, metadata);
         client.setNode(node);
@@ -514,10 +515,9 @@ public class KafkaConsumerTest {
                     @Override
                     public boolean matches(AbstractRequest body) {
                         ListOffsetRequest request = (ListOffsetRequest) body;
-                        Map<TopicPartition, Long> expectedTimestamps = new HashMap<>();
-                        expectedTimestamps.put(tp0, ListOffsetRequest.LATEST_TIMESTAMP);
-                        expectedTimestamps.put(tp1, ListOffsetRequest.EARLIEST_TIMESTAMP);
-                        return expectedTimestamps.equals(request.partitionTimestamps());
+                        Map<TopicPartition, ListOffsetRequest.PartitionData> timestamps = request.partitionTimestamps();
+                        return timestamps.get(tp0).timestamp == ListOffsetRequest.LATEST_TIMESTAMP &&
+                                timestamps.get(tp1).timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP;
                     }
                 }, listOffsetsResponse(Collections.singletonMap(tp0, 50L),
                         Collections.singletonMap(tp1, Errors.NOT_LEADER_FOR_PARTITION)));
@@ -1644,7 +1644,7 @@ public class KafkaConsumerTest {
 
     private JoinGroupResponse joinGroupFollowerResponse(PartitionAssignor assignor, int generationId, String memberId, String leaderId, Errors error) {
         return new JoinGroupResponse(error, generationId, assignor.name(), memberId, leaderId,
-                Collections.<String, ByteBuffer>emptyMap());
+                Collections.emptyMap());
     }
 
     private SyncGroupResponse syncGroupResponse(List<TopicPartition> partitions, Errors error) {
@@ -1655,7 +1655,8 @@ public class KafkaConsumerTest {
     private OffsetFetchResponse offsetResponse(Map<TopicPartition, Long> offsets, Errors error) {
         Map<TopicPartition, OffsetFetchResponse.PartitionData> partitionData = new HashMap<>();
         for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
-            partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(), "", error));
+            partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(),
+                    Optional.empty(), "", error));
         }
         return new OffsetFetchResponse(Errors.NONE, partitionData);
     }
@@ -1669,13 +1670,14 @@ public class KafkaConsumerTest {
         Map<TopicPartition, ListOffsetResponse.PartitionData> partitionData = new HashMap<>();
         for (Map.Entry<TopicPartition, Long> partitionOffset : partitionOffsets.entrySet()) {
             partitionData.put(partitionOffset.getKey(), new ListOffsetResponse.PartitionData(Errors.NONE,
-                    ListOffsetResponse.UNKNOWN_TIMESTAMP, partitionOffset.getValue()));
+                    ListOffsetResponse.UNKNOWN_TIMESTAMP, partitionOffset.getValue(),
+                    Optional.empty()));
         }
 
         for (Map.Entry<TopicPartition, Errors> partitionError : partitionErrors.entrySet()) {
             partitionData.put(partitionError.getKey(), new ListOffsetResponse.PartitionData(
                     partitionError.getValue(), ListOffsetResponse.UNKNOWN_TIMESTAMP,
-                    ListOffsetResponse.UNKNOWN_OFFSET));
+                    ListOffsetResponse.UNKNOWN_OFFSET, Optional.empty()));
         }
 
         return new ListOffsetResponse(partitionData);
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/OffsetAndMetadataTest.java
similarity index 50%
rename from clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java
rename to clients/src/test/java/org/apache/kafka/clients/consumer/OffsetAndMetadataTest.java
index 324aeaf..5bdbf7c 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/OffsetAndMetadataTest.java
@@ -20,9 +20,8 @@ import org.apache.kafka.common.utils.Serializer;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Optional;
 
-
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -30,35 +29,38 @@ import static org.junit.Assert.assertEquals;
  * Note: this ensures that the current code can deserialize data serialized with older versions of the code, but not the reverse.
  * That is, older code won't necessarily be able to deserialize data serialized with newer code.
  */
-public class SerializeCompatibilityOffsetAndMetadataTest {
-    private String metadata = "test commit metadata";
-    private String fileName = "serializedData/offsetAndMetadataSerializedfile";
-    private long offset = 10;
+public class OffsetAndMetadataTest {
 
-    private void checkValues(OffsetAndMetadata deSerOAM) {
-        //assert deserialized values are same as original
-        assertEquals("Offset should be " + offset + " but got " + deSerOAM.offset(), offset, deSerOAM.offset());
-        assertEquals("metadata should be " + metadata + " but got " + deSerOAM.metadata(), metadata, deSerOAM.metadata());
+    @Test(expected = IllegalArgumentException.class)
+    public void testInvalidNegativeOffset() {
+        new OffsetAndMetadata(-239L, Optional.of(15), "");
     }
 
     @Test
     public void testSerializationRoundtrip() throws IOException, ClassNotFoundException {
-        //assert OffsetAndMetadata is serializable
-        OffsetAndMetadata origOAM = new OffsetAndMetadata(offset, metadata);
-        byte[] byteArray =  Serializer.serialize(origOAM);
+        checkSerde(new OffsetAndMetadata(239L, Optional.of(15), "blah"));
+        checkSerde(new OffsetAndMetadata(239L, "blah"));
+        checkSerde(new OffsetAndMetadata(239L));
+    }
 
-        //deserialize the byteArray and check if the values are same as original
-        Object deserializedObject = Serializer.deserialize(byteArray);
-        assertTrue(deserializedObject instanceof OffsetAndMetadata);
-        checkValues((OffsetAndMetadata) deserializedObject);
+    private void checkSerde(OffsetAndMetadata offsetAndMetadata) throws IOException, ClassNotFoundException {
+        byte[] bytes =  Serializer.serialize(offsetAndMetadata);
+        OffsetAndMetadata deserialized = (OffsetAndMetadata) Serializer.deserialize(bytes);
+        assertEquals(offsetAndMetadata, deserialized);
     }
 
     @Test
-    public void testOffsetMetadataSerializationCompatibility() throws IOException, ClassNotFoundException {
-        // assert serialized OffsetAndMetadata object in file (oamserializedfile under resources folder) is
-        // deserializable into OffsetAndMetadata and is compatible
+    public void testDeserializationCompatibilityBeforeLeaderEpoch() throws IOException, ClassNotFoundException {
+        String fileName = "serializedData/offsetAndMetadataBeforeLeaderEpoch";
         Object deserializedObject = Serializer.deserialize(fileName);
-        assertTrue(deserializedObject instanceof OffsetAndMetadata);
-        checkValues((OffsetAndMetadata) deserializedObject);
+        assertEquals(new OffsetAndMetadata(10, "test commit metadata"), deserializedObject);
     }
+
+    @Test
+    public void testDeserializationCompatibilityWithLeaderEpoch() throws IOException, ClassNotFoundException {
+        String fileName = "serializedData/offsetAndMetadataWithLeaderEpoch";
+        Object deserializedObject = Serializer.deserialize(fileName);
+        assertEquals(new OffsetAndMetadata(10, Optional.of(235), "test commit metadata"), deserializedObject);
+    }
+
 }
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java
index 90e3b99..31cee7e 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java
@@ -711,8 +711,8 @@ public class StickyAssignorTest {
                 if (Math.abs(len - otherLen) <= 1)
                     continue;
 
-                Map<String, List<Integer>> map = CollectionUtils.groupDataByTopic(partitions);
-                Map<String, List<Integer>> otherMap = CollectionUtils.groupDataByTopic(otherPartitions);
+                Map<String, List<Integer>> map = CollectionUtils.groupPartitionsByTopic(partitions);
+                Map<String, List<Integer>> otherMap = CollectionUtils.groupPartitionsByTopic(otherPartitions);
 
                 if (len > otherLen) {
                     for (String topic: map.keySet())
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
index 2b6d303..af073f1 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
@@ -42,6 +42,7 @@ import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
 import org.apache.kafka.common.errors.WakeupException;
 import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.RecordBatch;
 import org.apache.kafka.common.requests.AbstractRequest;
 import org.apache.kafka.common.requests.FindCoordinatorResponse;
 import org.apache.kafka.common.requests.HeartbeatResponse;
@@ -72,6 +73,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -88,6 +90,7 @@ import static java.util.Collections.singletonList;
 import static java.util.Collections.singletonMap;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -245,7 +248,8 @@ public class ConsumerCoordinatorTest {
 
         final AtomicBoolean asyncCallbackInvoked = new AtomicBoolean(false);
         Map<TopicPartition, OffsetCommitRequest.PartitionData> offsets = singletonMap(
-                new TopicPartition("foo", 0), new OffsetCommitRequest.PartitionData(13L, ""));
+                new TopicPartition("foo", 0), new OffsetCommitRequest.PartitionData(13L,
+                        RecordBatch.NO_PARTITION_LEADER_EPOCH, ""));
         consumerClient.send(coordinator.checkAndGetCoordinator(), new OffsetCommitRequest.Builder(groupId, offsets))
                 .compose(new RequestFutureAdapter<ClientResponse, Object>() {
                     @Override
@@ -1571,22 +1575,6 @@ public class ConsumerCoordinatorTest {
         coordinator.commitOffsetsSync(singletonMap(t1p, new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE));
     }
 
-    @Test(expected = IllegalArgumentException.class)
-    public void testCommitSyncNegativeOffset() {
-        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
-        coordinator.commitOffsetsSync(singletonMap(t1p, new OffsetAndMetadata(-1L)), time.timer(Long.MAX_VALUE));
-    }
-
-    @Test
-    public void testCommitAsyncNegativeOffset() {
-        int invokedBeforeTest = mockOffsetCommitCallback.invoked;
-        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
-        coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(-1L)), mockOffsetCommitCallback);
-        coordinator.invokeCompletedOffsetCommitCallbacks();
-        assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked);
-        assertTrue(mockOffsetCommitCallback.exception instanceof IllegalArgumentException);
-    }
-
     @Test
     public void testCommitOffsetSyncWithoutFutureGetsCompleted() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
@@ -1609,6 +1597,25 @@ public class ConsumerCoordinatorTest {
     }
 
     @Test
+    public void testFetchCommittedOffsets() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        long offset = 500L;
+        String metadata = "blahblah";
+        Optional<Integer> leaderEpoch = Optional.of(15);
+        OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch,
+                metadata, Errors.NONE);
+
+        client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data)));
+        Map<TopicPartition, OffsetAndMetadata> fetchedOffsets = coordinator.fetchCommittedOffsets(singleton(t1p),
+                time.timer(Long.MAX_VALUE));
+
+        assertNotNull(fetchedOffsets);
+        assertEquals(new OffsetAndMetadata(offset, leaderEpoch, metadata), fetchedOffsets.get(t1p));
+    }
+
+    @Test
     public void testRefreshOffsetLoadInProgress() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
@@ -2074,7 +2081,8 @@ public class ConsumerCoordinatorTest {
     }
 
     private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) {
-        OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, metadata, partitionLevelError);
+        OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset,
+                Optional.empty(), metadata, partitionLevelError);
         return new OffsetFetchResponse(Errors.NONE, singletonMap(tp, data));
     }
 
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index fd550d61..d314a4d 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -62,11 +62,10 @@ import org.apache.kafka.common.record.RecordBatch;
 import org.apache.kafka.common.record.Records;
 import org.apache.kafka.common.record.SimpleRecord;
 import org.apache.kafka.common.record.TimestampType;
-import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.requests.AbstractRequest;
 import org.apache.kafka.common.requests.ApiVersionsResponse;
 import org.apache.kafka.common.requests.FetchRequest;
-import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.requests.IsolationLevel;
 import org.apache.kafka.common.requests.ListOffsetRequest;
 import org.apache.kafka.common.requests.ListOffsetResponse;
@@ -100,6 +99,7 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
 import static java.util.Collections.singleton;
@@ -209,6 +209,85 @@ public class FetcherTest {
     }
 
     @Test
+    public void testMissingLeaderEpochInRecords() {
+        subscriptions.assignFromUser(singleton(tp0));
+        subscriptions.seek(tp0, 0);
+
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0,
+                CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(),
+                RecordBatch.NO_PARTITION_LEADER_EPOCH);
+        builder.append(0L, "key".getBytes(), "1".getBytes());
+        builder.append(0L, "key".getBytes(), "2".getBytes());
+        MemoryRecords records = builder.build();
+
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        client.prepareResponse(fullFetchResponse(tp0, records, Errors.NONE, 100L, 0));
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+
+        Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetcher.fetchedRecords();
+        assertTrue(partitionRecords.containsKey(tp0));
+        assertEquals(2, partitionRecords.get(tp0).size());
+
+        for (ConsumerRecord<byte[], byte[]> record : partitionRecords.get(tp0)) {
+            assertEquals(Optional.empty(), record.leaderEpoch());
+        }
+    }
+
+    @Test
+    public void testLeaderEpochInConsumerRecord() {
+        subscriptions.assignFromUser(singleton(tp0));
+        subscriptions.seek(tp0, 0);
+
+        Integer partitionLeaderEpoch = 1;
+
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
+                CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(),
+                partitionLeaderEpoch);
+        builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes());
+        builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes());
+        builder.close();
+
+        partitionLeaderEpoch += 7;
+
+        builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+                TimestampType.CREATE_TIME, 2L, System.currentTimeMillis(), partitionLeaderEpoch);
+        builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes());
+        builder.close();
+
+        partitionLeaderEpoch += 5;
+        builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+                TimestampType.CREATE_TIME, 3L, System.currentTimeMillis(), partitionLeaderEpoch);
+        builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes());
+        builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes());
+        builder.append(0L, "key".getBytes(), partitionLeaderEpoch.toString().getBytes());
+        builder.close();
+
+        buffer.flip();
+        MemoryRecords records = MemoryRecords.readableRecords(buffer);
+
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        client.prepareResponse(fullFetchResponse(tp0, records, Errors.NONE, 100L, 0));
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+
+        Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetcher.fetchedRecords();
+        assertTrue(partitionRecords.containsKey(tp0));
+        assertEquals(6, partitionRecords.get(tp0).size());
+
+        for (ConsumerRecord<byte[], byte[]> record : partitionRecords.get(tp0)) {
+            int expectedLeaderEpoch = Integer.parseInt(Utils.utf8(record.value()));
+            assertEquals(Optional.of(expectedLeaderEpoch), record.leaderEpoch());
+        }
+    }
+
+    @Test
     public void testFetchSkipsBlackedOutNodes() {
         subscriptions.assignFromUser(singleton(tp0));
         subscriptions.seek(tp0, 0);
@@ -1403,6 +1482,7 @@ public class FetcherTest {
                     p.error(),
                     p.partition(),
                     null, //no leader
+                    Optional.empty(),
                     p.replicas(),
                     p.isr(),
                     p.offlineReplicas())
@@ -1843,9 +1923,11 @@ public class FetcherTest {
     public void testBatchedListOffsetsMetadataErrors() {
         Map<TopicPartition, ListOffsetResponse.PartitionData> partitionData = new HashMap<>();
         partitionData.put(tp0, new ListOffsetResponse.PartitionData(Errors.NOT_LEADER_FOR_PARTITION,
-                ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET));
+                ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET,
+                Optional.empty()));
         partitionData.put(tp1, new ListOffsetResponse.PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION,
-                ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET));
+                ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET,
+                Optional.empty()));
         client.prepareResponse(new ListOffsetResponse(0, partitionData));
 
         Map<TopicPartition, Long> offsetsToSearch = new HashMap<>();
@@ -2454,7 +2536,8 @@ public class FetcherTest {
 
         Map<TopicPartition, ListOffsetResponse.PartitionData> partitionData = new HashMap<>();
         partitionData.put(tp0, new ListOffsetResponse.PartitionData(Errors.NONE,
-                ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET));
+                ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET,
+                Optional.empty()));
 
         client.prepareResponseFrom(new ListOffsetResponse(0, partitionData), cluster.leaderFor(tp0));
 
@@ -2473,7 +2556,7 @@ public class FetcherTest {
             @Override
             public boolean matches(AbstractRequest body) {
                 ListOffsetRequest req = (ListOffsetRequest) body;
-                return timestamp == req.partitionTimestamps().get(tp0);
+                return timestamp == req.partitionTimestamps().get(tp0).timestamp;
             }
         };
     }
@@ -2483,7 +2566,8 @@ public class FetcherTest {
     }
 
     private ListOffsetResponse listOffsetResponse(TopicPartition tp, Errors error, long timestamp, long offset) {
-        ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error, timestamp, offset);
+        ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error, timestamp, offset,
+                Optional.empty());
         Map<TopicPartition, ListOffsetResponse.PartitionData> allPartitionData = new HashMap<>();
         allPartitionData.put(tp, partitionData);
         return new ListOffsetResponse(allPartitionData);
@@ -2526,6 +2610,7 @@ public class FetcherTest {
                         Errors.NONE,
                         partitionInfo.partition(),
                         partitionInfo.leader(),
+                        Optional.empty(),
                         Arrays.asList(partitionInfo.replicas()),
                         Arrays.asList(partitionInfo.inSyncReplicas()),
                         Arrays.asList(partitionInfo.offlineReplicas())));
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
index 606fa71..cf730b9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
@@ -2429,19 +2429,17 @@ public class TransactionManagerTest {
         }, new AddOffsetsToTxnResponse(0, error));
     }
 
-    private void prepareTxnOffsetCommitResponse(final String consumerGroupId, final long producerId,
-                                                final short producerEpoch, Map<TopicPartition, Errors> txnOffsetCommitResponse) {
-        client.prepareResponse(new MockClient.RequestMatcher() {
-            @Override
-            public boolean matches(AbstractRequest body) {
-                TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) body;
-                assertEquals(consumerGroupId, txnOffsetCommitRequest.consumerGroupId());
-                assertEquals(producerId, txnOffsetCommitRequest.producerId());
-                assertEquals(producerEpoch, txnOffsetCommitRequest.producerEpoch());
-                return true;
-            }
+    private void prepareTxnOffsetCommitResponse(final String consumerGroupId,
+                                                final long producerId,
+                                                final short producerEpoch,
+                                                Map<TopicPartition, Errors> txnOffsetCommitResponse) {
+        client.prepareResponse(request -> {
+            TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) request;
+            assertEquals(consumerGroupId, txnOffsetCommitRequest.consumerGroupId());
+            assertEquals(producerId, txnOffsetCommitRequest.producerId());
+            assertEquals(producerEpoch, txnOffsetCommitRequest.producerEpoch());
+            return true;
         }, new TxnOffsetCommitResponse(0, txnOffsetCommitResponse));
-
     }
 
     private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs) {
diff --git a/clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java b/clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java
similarity index 98%
rename from clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java
rename to clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java
index 8b4df5f..2a90338 100644
--- a/clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java
@@ -29,8 +29,7 @@ import static org.junit.Assert.assertEquals;
  * Note: this ensures that the current code can deserialize data serialized with older versions of the code, but not the reverse.
  * That is, older code won't necessarily be able to deserialize data serialized with newer code.
  */
-public class SerializeCompatibilityTopicPartitionTest {
-
+public class TopicPartitionTest {
     private String topicName = "mytopic";
     private String fileName = "serializedData/topicPartitionSerializedfile";
     private int partNum = 5;
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index da61398..05b9926 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -72,6 +72,7 @@ import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
 import static java.util.Arrays.asList;
@@ -469,14 +470,15 @@ public class RequestResponseTest {
 
     @Test
     public void fetchResponseVersionTest() {
-        LinkedHashMap<TopicPartition, FetchResponse.PartitionData> responseData = new LinkedHashMap<>();
+        LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>();
 
         MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10));
-        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE, 1000000,
-                FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, records));
+        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData<>(
+                Errors.NONE, 1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET,
+                0L, null, records));
 
-        FetchResponse v0Response = new FetchResponse(Errors.NONE, responseData, 0, INVALID_SESSION_ID);
-        FetchResponse v1Response = new FetchResponse(Errors.NONE, responseData, 10, INVALID_SESSION_ID);
+        FetchResponse<MemoryRecords> v0Response = new FetchResponse<>(Errors.NONE, responseData, 0, INVALID_SESSION_ID);
+        FetchResponse<MemoryRecords> v1Response = new FetchResponse<>(Errors.NONE, responseData, 10, INVALID_SESSION_ID);
         assertEquals("Throttle time must be zero", 0, v0Response.throttleTimeMs());
         assertEquals("Throttle time must be 10", 10, v1Response.throttleTimeMs());
         assertEquals("Should use schema version 0", ApiKeys.FETCH.responseSchema((short) 0),
@@ -489,22 +491,21 @@ public class RequestResponseTest {
 
     @Test
     public void testFetchResponseV4() {
-        LinkedHashMap<TopicPartition, FetchResponse.PartitionData> responseData = new LinkedHashMap<>();
-
+        LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>();
         MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10));
 
         List<FetchResponse.AbortedTransaction> abortedTransactions = asList(
                 new FetchResponse.AbortedTransaction(10, 100),
                 new FetchResponse.AbortedTransaction(15, 50)
         );
-        responseData.put(new TopicPartition("bar", 0), new FetchResponse.PartitionData(Errors.NONE, 100000,
+        responseData.put(new TopicPartition("bar", 0), new FetchResponse.PartitionData<>(Errors.NONE, 100000,
                 FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, abortedTransactions, records));
-        responseData.put(new TopicPartition("bar", 1), new FetchResponse.PartitionData(Errors.NONE, 900000,
+        responseData.put(new TopicPartition("bar", 1), new FetchResponse.PartitionData<>(Errors.NONE, 900000,
                 5, FetchResponse.INVALID_LOG_START_OFFSET, null, records));
-        responseData.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData(Errors.NONE, 70000,
-                6, FetchResponse.INVALID_LOG_START_OFFSET, Collections.<FetchResponse.AbortedTransaction>emptyList(), records));
+        responseData.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData<>(Errors.NONE, 70000,
+                6, FetchResponse.INVALID_LOG_START_OFFSET, Collections.emptyList(), records));
 
-        FetchResponse response = new FetchResponse(Errors.NONE, responseData, 10, INVALID_SESSION_ID);
+        FetchResponse<MemoryRecords> response = new FetchResponse<>(Errors.NONE, responseData, 10, INVALID_SESSION_ID);
         FetchResponse deserialized = FetchResponse.parse(toBuffer(response.toStruct((short) 4)), (short) 4);
         assertEquals(responseData, deserialized.responseData());
     }
@@ -595,7 +596,7 @@ public class RequestResponseTest {
     }
 
     @Test
-    public void testJoinGroupRequestVersion0RebalanceTimeout() throws Exception {
+    public void testJoinGroupRequestVersion0RebalanceTimeout() {
         final short version = 0;
         JoinGroupRequest jgr = createJoinGroupRequest(version);
         JoinGroupRequest jgr2 = new JoinGroupRequest(jgr.toStruct(), version);
@@ -627,56 +628,61 @@ public class RequestResponseTest {
 
     private FetchRequest createFetchRequest(int version, FetchMetadata metadata, List<TopicPartition> toForget) {
         LinkedHashMap<TopicPartition, FetchRequest.PartitionData> fetchData = new LinkedHashMap<>();
-        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 0L, 1000000));
-        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 0L, 1000000));
+        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 0L,
+                1000000, Optional.of(15)));
+        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 0L,
+                1000000, Optional.of(25)));
         return FetchRequest.Builder.forConsumer(100, 100000, fetchData).
             metadata(metadata).setMaxBytes(1000).toForget(toForget).build((short) version);
     }
 
     private FetchRequest createFetchRequest(int version, IsolationLevel isolationLevel) {
         LinkedHashMap<TopicPartition, FetchRequest.PartitionData> fetchData = new LinkedHashMap<>();
-        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 0L, 1000000));
-        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 0L, 1000000));
+        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 0L,
+                1000000, Optional.of(15)));
+        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 0L,
+                1000000, Optional.of(25)));
         return FetchRequest.Builder.forConsumer(100, 100000, fetchData).
             isolationLevel(isolationLevel).setMaxBytes(1000).build((short) version);
     }
 
     private FetchRequest createFetchRequest(int version) {
         LinkedHashMap<TopicPartition, FetchRequest.PartitionData> fetchData = new LinkedHashMap<>();
-        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 0L, 1000000));
-        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 0L, 1000000));
+        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 0L,
+                1000000, Optional.of(15)));
+        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 0L,
+                1000000, Optional.of(25)));
         return FetchRequest.Builder.forConsumer(100, 100000, fetchData).setMaxBytes(1000).build((short) version);
     }
 
-    private FetchResponse createFetchResponse(Errors error, int sessionId) {
-        return new FetchResponse(error, new LinkedHashMap<TopicPartition, FetchResponse.PartitionData>(),
-            25, sessionId);
+    private FetchResponse<MemoryRecords> createFetchResponse(Errors error, int sessionId) {
+        return new FetchResponse<>(error, new LinkedHashMap<>(), 25, sessionId);
     }
 
-    private FetchResponse createFetchResponse(int sessionId) {
-        LinkedHashMap<TopicPartition, FetchResponse.PartitionData> responseData = new LinkedHashMap<>();
+    private FetchResponse<MemoryRecords> createFetchResponse(int sessionId) {
+        LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>();
         MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes()));
-        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE,
+        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData<>(Errors.NONE,
             1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, records));
         List<FetchResponse.AbortedTransaction> abortedTransactions = Collections.singletonList(
             new FetchResponse.AbortedTransaction(234L, 999L));
-        responseData.put(new TopicPartition("test", 1), new FetchResponse.PartitionData(Errors.NONE,
+        responseData.put(new TopicPartition("test", 1), new FetchResponse.PartitionData<>(Errors.NONE,
             1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, abortedTransactions, MemoryRecords.EMPTY));
-        return new FetchResponse(Errors.NONE, responseData, 25, sessionId);
+        return new FetchResponse<>(Errors.NONE, responseData, 25, sessionId);
     }
 
-    private FetchResponse createFetchResponse() {
-        LinkedHashMap<TopicPartition, FetchResponse.PartitionData> responseData = new LinkedHashMap<>();
+    private FetchResponse<MemoryRecords> createFetchResponse() {
+        LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>();
         MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes()));
-        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE,
+        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData<>(Errors.NONE,
                 1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, records));
 
         List<FetchResponse.AbortedTransaction> abortedTransactions = Collections.singletonList(
                 new FetchResponse.AbortedTransaction(234L, 999L));
-        responseData.put(new TopicPartition("test", 1), new FetchResponse.PartitionData(Errors.NONE,
+        responseData.put(new TopicPartition("test", 1), new FetchResponse.PartitionData<>(Errors.NONE,
                 1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, abortedTransactions, MemoryRecords.EMPTY));
 
-        return new FetchResponse(Errors.NONE, responseData, 25, INVALID_SESSION_ID);
+        return new FetchResponse<>(Errors.NONE, responseData, 25, INVALID_SESSION_ID);
     }
 
     private HeartbeatRequest createHeartBeatRequest() {
@@ -757,18 +763,20 @@ public class RequestResponseTest {
                     new ListOffsetRequest.PartitionData(1000000L, 10));
             return ListOffsetRequest.Builder
                     .forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
-                    .setOffsetData(offsetData)
+                    .setTargetTimes(offsetData)
                     .build((short) version);
         } else if (version == 1) {
-            Map<TopicPartition, Long> offsetData = Collections.singletonMap(
-                    new TopicPartition("test", 0), 1000000L);
+            Map<TopicPartition, ListOffsetRequest.PartitionData> offsetData = Collections.singletonMap(
+                    new TopicPartition("test", 0),
+                    new ListOffsetRequest.PartitionData(1000000L, Optional.empty()));
             return ListOffsetRequest.Builder
                     .forConsumer(true, IsolationLevel.READ_UNCOMMITTED)
                     .setTargetTimes(offsetData)
                     .build((short) version);
         } else if (version == 2) {
-            Map<TopicPartition, Long> offsetData = Collections.singletonMap(
-                    new TopicPartition("test", 0), 1000000L);
+            Map<TopicPartition, ListOffsetRequest.PartitionData> offsetData = Collections.singletonMap(
+                    new TopicPartition("test", 0),
+                    new ListOffsetRequest.PartitionData(1000000L, Optional.of(5)));
             return ListOffsetRequest.Builder
                     .forConsumer(true, IsolationLevel.READ_COMMITTED)
                     .setTargetTimes(offsetData)
@@ -788,7 +796,7 @@ public class RequestResponseTest {
         } else if (version == 1 || version == 2) {
             Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<>();
             responseData.put(new TopicPartition("test", 0),
-                    new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L));
+                    new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L, Optional.of(27)));
             return new ListOffsetResponse(responseData);
         } else {
             throw new IllegalArgumentException("Illegal ListOffsetResponse version " + version);
@@ -807,20 +815,23 @@ public class RequestResponseTest {
 
         List<MetadataResponse.TopicMetadata> allTopicMetadata = new ArrayList<>();
         allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "__consumer_offsets", true,
-                asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr, offlineReplicas))));
+                asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node,
+                        Optional.of(5), replicas, isr, offlineReplicas))));
         allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false,
-                Collections.<MetadataResponse.PartitionMetadata>emptyList()));
+                Collections.emptyList()));
         allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "topic3", false,
             asList(new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, 0, null,
-                replicas, isr, offlineReplicas))));
+                Optional.empty(), replicas, isr, offlineReplicas))));
 
         return new MetadataResponse(asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata);
     }
 
     private OffsetCommitRequest createOffsetCommitRequest(int version) {
         Map<TopicPartition, OffsetCommitRequest.PartitionData> commitData = new HashMap<>();
-        commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, ""));
-        commitData.put(new TopicPartition("test", 1), new OffsetCommitRequest.PartitionData(200, null));
+        commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100,
+                RecordBatch.NO_PARTITION_LEADER_EPOCH, ""));
+        commitData.put(new TopicPartition("test", 1), new OffsetCommitRequest.PartitionData(200,
+                RecordBatch.NO_PARTITION_LEADER_EPOCH, null));
         return new OffsetCommitRequest.Builder("group1", commitData)
                 .setGenerationId(100)
                 .setMemberId("consumer1")
@@ -840,8 +851,10 @@ public class RequestResponseTest {
 
     private OffsetFetchResponse createOffsetFetchResponse() {
         Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<>();
-        responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE));
-        responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData(100L, null, Errors.NONE));
+        responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(
+                100L, Optional.empty(), "", Errors.NONE));
+        responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData(
+                100L, Optional.of(10), null, Errors.NONE));
         return new OffsetFetchResponse(Errors.NONE, responseData);
     }
 
@@ -1021,11 +1034,14 @@ public class RequestResponseTest {
 
 
     private OffsetsForLeaderEpochRequest createLeaderEpochRequest() {
-        Map<TopicPartition, Integer> epochs = new HashMap<>();
+        Map<TopicPartition, OffsetsForLeaderEpochRequest.PartitionData> epochs = new HashMap<>();
 
-        epochs.put(new TopicPartition("topic1", 0), 1);
-        epochs.put(new TopicPartition("topic1", 1), 1);
-        epochs.put(new TopicPartition("topic2", 2), 3);
+        epochs.put(new TopicPartition("topic1", 0),
+                new OffsetsForLeaderEpochRequest.PartitionData(Optional.of(0), 1));
+        epochs.put(new TopicPartition("topic1", 1),
+                new OffsetsForLeaderEpochRequest.PartitionData(Optional.of(0), 1));
+        epochs.put(new TopicPartition("topic2", 2),
+                new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), 3));
 
         return new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(), epochs).build();
     }
@@ -1082,7 +1098,9 @@ public class RequestResponseTest {
     private TxnOffsetCommitRequest createTxnOffsetCommitRequest() {
         final Map<TopicPartition, TxnOffsetCommitRequest.CommittedOffset> offsets = new HashMap<>();
         offsets.put(new TopicPartition("topic", 73),
-                    new TxnOffsetCommitRequest.CommittedOffset(100, null));
+                    new TxnOffsetCommitRequest.CommittedOffset(100, null, Optional.empty()));
+        offsets.put(new TopicPartition("topic", 74),
+                new TxnOffsetCommitRequest.CommittedOffset(100, "blah", Optional.of(27)));
         return new TxnOffsetCommitRequest.Builder("transactionalId", "groupId", 21L, (short) 42, offsets).build();
     }
 
diff --git a/clients/src/test/resources/serializedData/offsetAndMetadataSerializedfile b/clients/src/test/resources/serializedData/offsetAndMetadataBeforeLeaderEpoch
similarity index 100%
rename from clients/src/test/resources/serializedData/offsetAndMetadataSerializedfile
rename to clients/src/test/resources/serializedData/offsetAndMetadataBeforeLeaderEpoch
diff --git a/clients/src/test/resources/serializedData/offsetAndMetadataWithLeaderEpoch b/clients/src/test/resources/serializedData/offsetAndMetadataWithLeaderEpoch
new file mode 100644
index 0000000..ddf3956
Binary files /dev/null and b/clients/src/test/resources/serializedData/offsetAndMetadataWithLeaderEpoch differ
diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala
index fd37367..b145adf 100644
--- a/core/src/main/scala/kafka/api/ApiVersion.scala
+++ b/core/src/main/scala/kafka/api/ApiVersion.scala
@@ -75,7 +75,9 @@ object ApiVersion {
     // Several request versions were bumped due to KIP-219 (Improve quota communication)
     KAFKA_2_0_IV1,
     // Introduced new schemas for group offset (v2) and group metadata (v2) (KIP-211)
-    KAFKA_2_1_IV0
+    KAFKA_2_1_IV0,
+    // New Fetch, OffsetsForLeaderEpoch, and ListOffsets schemas (KIP-320)
+    KAFKA_2_1_IV1
   )
 
   // Map keys are the union of the short and full versions
@@ -256,5 +258,12 @@ case object KAFKA_2_1_IV0 extends DefaultApiVersion {
   val shortVersion: String = "2.1"
   val subVersion = "IV0"
   val recordVersion = RecordVersion.V2
+  val id: Int = 17
+}
+
+case object KAFKA_2_1_IV1 extends DefaultApiVersion {
+  val shortVersion: String = "2.1"
+  val subVersion = "IV1"
+  val recordVersion = RecordVersion.V2
   val id: Int = 18
 }
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
index 940ec71..8cf99fc 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
@@ -20,6 +20,7 @@ package kafka.coordinator.group
 import java.io.PrintStream
 import java.nio.ByteBuffer
 import java.nio.charset.StandardCharsets
+import java.util.Optional
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic.AtomicBoolean
 import java.util.concurrent.locks.ReentrantLock
@@ -440,13 +441,17 @@ class GroupMetadataManager(brokerId: Int,
     val group = groupMetadataCache.get(groupId)
     if (group == null) {
       topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition =>
-        (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE))
+        val partitionData = new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
+          Optional.empty(), "", Errors.NONE)
+        topicPartition -> partitionData
       }.toMap
     } else {
       group.inLock {
         if (group.is(Dead)) {
           topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition =>
-            (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE))
+            val partitionData = new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
+              Optional.empty(), "", Errors.NONE)
+            topicPartition -> partitionData
           }.toMap
         } else {
           topicPartitionsOpt match {
@@ -454,16 +459,19 @@ class GroupMetadataManager(brokerId: Int,
               // Return offsets for all partitions owned by this consumer group. (this only applies to consumers
               // that commit offsets to Kafka.)
               group.allOffsets.map { case (topicPartition, offsetAndMetadata) =>
-                topicPartition -> new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)
+                topicPartition -> new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset,
+                  Optional.empty(), offsetAndMetadata.metadata, Errors.NONE)
               }
 
             case Some(topicPartitions) =>
               topicPartitions.map { topicPartition =>
                 val partitionData = group.offset(topicPartition) match {
                   case None =>
-                    new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE)
+                    new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
+                      Optional.empty(), "", Errors.NONE)
                   case Some(offsetAndMetadata) =>
-                    new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)
+                    new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset,
+                      Optional.empty(), offsetAndMetadata.metadata, Errors.NONE)
                 }
                 topicPartition -> partitionData
               }.toMap
diff --git a/core/src/main/scala/kafka/server/FetchSession.scala b/core/src/main/scala/kafka/server/FetchSession.scala
index 64bc773..16ee872 100644
--- a/core/src/main/scala/kafka/server/FetchSession.scala
+++ b/core/src/main/scala/kafka/server/FetchSession.scala
@@ -18,6 +18,7 @@
 package kafka.server
 
 import java.util
+import java.util.Optional
 import java.util.concurrent.{ThreadLocalRandom, TimeUnit}
 
 import com.yammer.metrics.core.Gauge
@@ -107,7 +108,7 @@ class CachedPartition(val topic: String,
 
   def topicPartition = new TopicPartition(topic, partition)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes)
+  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, Optional.empty())
 
   def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
     // Update our cached request parameters.
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 3a81b89..24c13cd 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer
 import java.util
 import java.util.concurrent.ConcurrentHashMap
 import java.util.concurrent.atomic.AtomicInteger
-import java.util.{Collections, Properties}
+import java.util.{Collections, Optional, Properties}
 
 import kafka.admin.{AdminUtils, RackAwareMode}
 import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0}
@@ -735,7 +735,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     val clientId = request.header.clientId
     val offsetRequest = request.body[ListOffsetRequest]
 
-    val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.offsetData.asScala.partition {
+    val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.partitionTimestamps.asScala.partition {
       case (topicPartition, _) => authorize(request.session, Describe, Resource(Topic, topicPartition.topic, LITERAL))
     }
 
@@ -794,17 +794,19 @@ class KafkaApis(val requestChannel: RequestChannel,
 
     val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => {
       new ListOffsetResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED,
-                                           ListOffsetResponse.UNKNOWN_TIMESTAMP,
-                                           ListOffsetResponse.UNKNOWN_OFFSET)
+        ListOffsetResponse.UNKNOWN_TIMESTAMP,
+        ListOffsetResponse.UNKNOWN_OFFSET,
+        Optional.empty())
     })
 
-    val responseMap = authorizedRequestInfo.map { case (topicPartition, timestamp) =>
+    val responseMap = authorizedRequestInfo.map { case (topicPartition, partitionData) =>
       if (offsetRequest.duplicatePartitions().contains(topicPartition)) {
         debug(s"OffsetRequest with correlation id $correlationId from client $clientId on partition $topicPartition " +
             s"failed because the partition is duplicated in the request.")
         (topicPartition, new ListOffsetResponse.PartitionData(Errors.INVALID_REQUEST,
-                                                              ListOffsetResponse.UNKNOWN_TIMESTAMP,
-                                                              ListOffsetResponse.UNKNOWN_OFFSET))
+          ListOffsetResponse.UNKNOWN_TIMESTAMP,
+          ListOffsetResponse.UNKNOWN_OFFSET,
+          Optional.empty()))
       } else {
         try {
           // ensure leader exists
@@ -820,21 +822,22 @@ class KafkaApis(val requestChannel: RequestChannel,
               case IsolationLevel.READ_UNCOMMITTED => localReplica.highWatermark.messageOffset
             }
 
-            if (timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
+            if (partitionData.timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
               TimestampOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset)
             else {
               def allowed(timestampOffset: TimestampOffset): Boolean =
-                timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP || timestampOffset.offset < lastFetchableOffset
+                partitionData.timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP || timestampOffset.offset < lastFetchableOffset
 
-              fetchOffsetForTimestamp(topicPartition, timestamp)
+              fetchOffsetForTimestamp(topicPartition, partitionData.timestamp)
                 .filter(allowed).getOrElse(TimestampOffset.Unknown)
             }
           } else {
-            fetchOffsetForTimestamp(topicPartition, timestamp)
+            fetchOffsetForTimestamp(topicPartition, partitionData.timestamp)
               .getOrElse(TimestampOffset.Unknown)
           }
 
-          (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE, found.timestamp, found.offset))
+          (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE, found.timestamp, found.offset,
+            Optional.empty()))
         } catch {
           // NOTE: These exceptions are special cased since these error messages are typically transient or the client
           // would have received a clear exception and there is no value in logging the entire stack trace for the same
@@ -845,13 +848,15 @@ class KafkaApis(val requestChannel: RequestChannel,
             debug(s"Offset request with correlation id $correlationId from client $clientId on " +
                 s"partition $topicPartition failed due to ${e.getMessage}")
             (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(e),
-                                                                  ListOffsetResponse.UNKNOWN_TIMESTAMP,
-                                                                  ListOffsetResponse.UNKNOWN_OFFSET))
+              ListOffsetResponse.UNKNOWN_TIMESTAMP,
+              ListOffsetResponse.UNKNOWN_OFFSET,
+              Optional.empty()))
           case e: Throwable =>
             error("Error while responding to offset request", e)
             (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(e),
-                                                                  ListOffsetResponse.UNKNOWN_TIMESTAMP,
-                                                                  ListOffsetResponse.UNKNOWN_OFFSET))
+              ListOffsetResponse.UNKNOWN_TIMESTAMP,
+              ListOffsetResponse.UNKNOWN_OFFSET,
+              Optional.empty()))
         }
       }
     }
@@ -1118,16 +1123,16 @@ class KafkaApis(val requestChannel: RequestChannel,
                   val payloadOpt = zkClient.getConsumerOffset(offsetFetchRequest.groupId, topicPartition)
                   payloadOpt match {
                     case Some(payload) =>
-                      (topicPartition, new OffsetFetchResponse.PartitionData(
-                          payload.toLong, OffsetFetchResponse.NO_METADATA, Errors.NONE))
+                      (topicPartition, new OffsetFetchResponse.PartitionData(payload.toLong,
+                        Optional.empty(), OffsetFetchResponse.NO_METADATA, Errors.NONE))
                     case None =>
                       (topicPartition, OffsetFetchResponse.UNKNOWN_PARTITION)
                   }
                 }
               } catch {
                 case e: Throwable =>
-                  (topicPartition, new OffsetFetchResponse.PartitionData(
-                      OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, Errors.forException(e)))
+                  (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET,
+                    Optional.empty(), OffsetFetchResponse.NO_METADATA, Errors.forException(e)))
               }
             }.toMap
 
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 25967b3..3114663 100755
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -17,7 +17,7 @@
 
 package kafka.server
 
-import java.util.Collections
+import java.util.{Collections, Optional}
 import java.util.concurrent.locks.ReentrantReadWriteLock
 
 import scala.collection.{Seq, Set, mutable}
@@ -74,6 +74,7 @@ class MetadataCache(brokerId: Int) extends Logging {
       partitions.map { case (partitionId, partitionState) =>
         val topicPartition = new TopicPartition(topic, partitionId.toInt)
         val leaderBrokerId = partitionState.basePartitionState.leader
+        val leaderEpoch = partitionState.basePartitionState.leaderEpoch
         val maybeLeader = getAliveEndpoint(snapshot, leaderBrokerId, listenerName)
         val replicas = partitionState.basePartitionState.replicas.asScala.map(_.toInt)
         val replicaInfo = getEndpoints(snapshot, replicas, listenerName, errorUnavailableEndpoints)
@@ -89,7 +90,8 @@ class MetadataCache(brokerId: Int) extends Logging {
               if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE
             }
             new MetadataResponse.PartitionMetadata(error, partitionId.toInt, Node.noNode(),
-              replicaInfo.asJava, java.util.Collections.emptyList(), offlineReplicaInfo.asJava)
+              Optional.empty(), replicaInfo.asJava, java.util.Collections.emptyList(),
+              offlineReplicaInfo.asJava)
 
           case Some(leader) =>
             val isr = partitionState.basePartitionState.isr.asScala.map(_.toInt)
@@ -100,15 +102,15 @@ class MetadataCache(brokerId: Int) extends Logging {
                 s"following brokers ${replicas.filterNot(replicaInfo.map(_.id).contains).mkString(",")}")
 
               new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId.toInt, leader,
-                replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
+                Optional.empty(), replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
             } else if (isrInfo.size < isr.size) {
               debug(s"Error while fetching metadata for $topicPartition: in sync replica information not available for " +
                 s"following brokers ${isr.filterNot(isrInfo.map(_.id).contains).mkString(",")}")
               new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId.toInt, leader,
-                replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
+                Optional.empty(), replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
             } else {
-              new MetadataResponse.PartitionMetadata(Errors.NONE, partitionId.toInt, leader, replicaInfo.asJava,
-                isrInfo.asJava, offlineReplicaInfo.asJava)
+              new MetadataResponse.PartitionMetadata(Errors.NONE, partitionId.toInt, leader, Optional.of(leaderEpoch),
+                replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
             }
         }
       }
diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
index 5aec7a9..dc585eb 100644
--- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
@@ -18,6 +18,7 @@
 package kafka.server
 
 import java.util
+import java.util.Optional
 
 import kafka.api.Request
 import kafka.cluster.BrokerEndPoint
@@ -196,7 +197,8 @@ class ReplicaAlterLogDirsThread(name: String,
       val (topicPartition, partitionFetchState) = maxPartitionOpt.get
       try {
         val logStartOffset = replicaMgr.getReplicaOrException(topicPartition, Request.FutureLocalReplicaId).logStartOffset
-        requestMap.put(topicPartition, new FetchRequest.PartitionData(partitionFetchState.fetchOffset, logStartOffset, fetchSize))
+        requestMap.put(topicPartition, new FetchRequest.PartitionData(partitionFetchState.fetchOffset, logStartOffset,
+          fetchSize, Optional.empty()))
       } catch {
         case _: KafkaStorageException =>
           partitionsWithError += topicPartition
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index 1848eb7..5dcd29b 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -17,6 +17,8 @@
 
 package kafka.server
 
+import java.util.Optional
+
 import kafka.api._
 import kafka.cluster.BrokerEndPoint
 import kafka.log.{LogAppendInfo, LogConfig}
@@ -61,7 +63,8 @@ class ReplicaFetcherThread(name: String,
 
   // Visible for testing
   private[server] val fetchRequestVersion: Short =
-    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 8
+    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 9
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 8
     else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_1_1_IV0) 7
     else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV1) 5
     else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 4
@@ -72,12 +75,14 @@ class ReplicaFetcherThread(name: String,
 
   // Visible for testing
   private[server] val offsetForLeaderEpochRequestVersion: Short =
-    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV0) 1
+    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 2
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV0) 1
     else 0
 
   // Visible for testing
   private[server] val listOffsetRequestVersion: Short =
-    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 3
+    if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 4
+    else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 3
     else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 2
     else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) 1
     else 0
@@ -199,22 +204,21 @@ class ReplicaFetcherThread(name: String,
   }
 
   private def fetchOffsetFromLeader(topicPartition: TopicPartition, earliestOrLatest: Long): Long = {
-    val requestBuilder = if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) {
-        val partitions = Map(topicPartition -> (earliestOrLatest: java.lang.Long))
-        ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId).setTargetTimes(partitions.asJava)
-      } else {
-        val partitions = Map(topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1))
-        ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId).setOffsetData(partitions.asJava)
-      }
+    val requestPartitionData = new ListOffsetRequest.PartitionData(earliestOrLatest, Optional.empty[Integer]())
+    val requestPartitions = Map(topicPartition -> requestPartitionData)
+    val requestBuilder = ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId)
+      .setTargetTimes(requestPartitions.asJava)
+
     val clientResponse = leaderEndpoint.sendRequest(requestBuilder)
     val response = clientResponse.responseBody.asInstanceOf[ListOffsetResponse]
-    val partitionData = response.responseData.get(topicPartition)
-    partitionData.error match {
+
+    val responsePartitionData = response.responseData.get(topicPartition)
+    responsePartitionData.error match {
       case Errors.NONE =>
         if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2)
-          partitionData.offset
+          responsePartitionData.offset
         else
-          partitionData.offsets.get(0)
+          responsePartitionData.offsets.get(0)
       case error => throw error.exception
     }
   }
@@ -229,7 +233,7 @@ class ReplicaFetcherThread(name: String,
         try {
           val logStartOffset = replicaMgr.getReplicaOrException(topicPartition).logStartOffset
           builder.add(topicPartition, new FetchRequest.PartitionData(
-            partitionFetchState.fetchOffset, logStartOffset, fetchSize))
+            partitionFetchState.fetchOffset, logStartOffset, fetchSize, Optional.empty()))
         } catch {
           case _: KafkaStorageException =>
             // The replica has already been marked offline due to log directory failure and the original failure should have already been logged.
@@ -289,7 +293,9 @@ class ReplicaFetcherThread(name: String,
         return resultWithoutEpoch
       }
 
-      val partitionsAsJava = partitionsWithEpoch.map { case (tp, epoch) => tp -> epoch.asInstanceOf[Integer] }.toMap.asJava
+      val partitionsAsJava = partitions.map { case (tp, epoch) => tp ->
+        new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), epoch.asInstanceOf[Integer])
+      }.toMap.asJava
       val epochRequest = new OffsetsForLeaderEpochRequest.Builder(offsetForLeaderEpochRequestVersion, partitionsAsJava)
       try {
         val response = leaderEndpoint.sendRequest(epochRequest)
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 2393daa..d61240e 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1475,14 +1475,14 @@ class ReplicaManager(val config: KafkaConfig,
     new ReplicaAlterLogDirsManager(config, this, quotaManager, brokerTopicStats)
   }
 
-  def lastOffsetForLeaderEpoch(requestedEpochInfo: Map[TopicPartition, Integer]): Map[TopicPartition, EpochEndOffset] = {
-    requestedEpochInfo.map { case (tp, leaderEpoch) =>
+  def lastOffsetForLeaderEpoch(requestedEpochInfo: Map[TopicPartition, OffsetsForLeaderEpochRequest.PartitionData]): Map[TopicPartition, EpochEndOffset] = {
+    requestedEpochInfo.map { case (tp, partitionData) =>
       val epochEndOffset = getPartition(tp) match {
         case Some(partition) =>
           if (partition eq ReplicaManager.OfflinePartition)
             new EpochEndOffset(Errors.KAFKA_STORAGE_ERROR, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
           else
-            partition.lastOffsetForLeaderEpoch(leaderEpoch)
+            partition.lastOffsetForLeaderEpoch(partitionData.leaderEpoch)
 
         case None if metadataCache.contains(tp) =>
           new EpochEndOffset(Errors.NOT_LEADER_FOR_PARTITION, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
index 1ecea09..4758764 100644
--- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
+++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
@@ -23,7 +23,7 @@ import java.util
 import java.util.concurrent.CountDownLatch
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
 import java.util.regex.{Pattern, PatternSyntaxException}
-import java.util.{Date, Properties}
+import java.util.{Date, Optional, Properties}
 
 import joptsimple.OptionParser
 import kafka.api._
@@ -389,7 +389,8 @@ private class ReplicaFetcher(name: String, sourceBroker: Node, topicPartitions:
 
     val requestMap = new util.LinkedHashMap[TopicPartition, JFetchRequest.PartitionData]
     for (topicPartition <- topicPartitions)
-      requestMap.put(topicPartition, new JFetchRequest.PartitionData(replicaBuffer.getOffset(topicPartition), 0L, fetchSize))
+      requestMap.put(topicPartition, new JFetchRequest.PartitionData(replicaBuffer.getOffset(topicPartition),
+        0L, fetchSize, Optional.empty()))
 
     val fetchRequestBuilder = JFetchRequest.Builder.
       forReplica(ApiKeys.FETCH.latestVersion, Request.DebuggingConsumerId, maxWait, minBytes, requestMap)
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index 7c341e6..cfc6b5e 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -16,7 +16,7 @@ import java.nio.ByteBuffer
 import java.util
 import java.util.concurrent.ExecutionException
 import java.util.regex.Pattern
-import java.util.{ArrayList, Collections, Properties}
+import java.util.{ArrayList, Collections, Optional, Properties}
 import java.time.Duration
 
 import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService}
@@ -260,25 +260,26 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
 
   private def createFetchRequest = {
     val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData]
-    partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 0, 100))
+    partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 0, 100, Optional.of(27)))
     requests.FetchRequest.Builder.forConsumer(100, Int.MaxValue, partitionMap).build()
   }
 
   private def createFetchFollowerRequest = {
     val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData]
-    partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 0, 100))
+    partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 0, 100, Optional.of(27)))
     val version = ApiKeys.FETCH.latestVersion
     requests.FetchRequest.Builder.forReplica(version, 5000, 100, Int.MaxValue, partitionMap).build()
   }
 
   private def createListOffsetsRequest = {
     requests.ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED).setTargetTimes(
-      Map(tp -> (0L: java.lang.Long)).asJava).
+      Map(tp -> new ListOffsetRequest.PartitionData(0L, Optional.of[Integer](27))).asJava).
       build()
   }
 
   private def offsetsForLeaderEpochRequest = {
-    new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion()).add(tp, 7).build()
+    new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion)
+      .add(tp, Optional.of(27), 7).build()
   }
 
   private def createOffsetFetchRequest = {
@@ -316,7 +317,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
 
   private def createOffsetCommitRequest = {
     new requests.OffsetCommitRequest.Builder(
-      group, Map(tp -> new requests.OffsetCommitRequest.PartitionData(0, "metadata")).asJava).
+      group, Map(tp -> new requests.OffsetCommitRequest.PartitionData(0, 27, "metadata")).asJava).
       setMemberId("").setGenerationId(1).
       build()
   }
diff --git a/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala b/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala
index 2befc8f..d2d115b 100644
--- a/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala
+++ b/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala
@@ -80,8 +80,9 @@ class ApiVersionTest {
     assertEquals(KAFKA_2_0_IV0, ApiVersion("2.0-IV0"))
     assertEquals(KAFKA_2_0_IV1, ApiVersion("2.0-IV1"))
 
-    assertEquals(KAFKA_2_1_IV0, ApiVersion("2.1"))
+    assertEquals(KAFKA_2_1_IV1, ApiVersion("2.1"))
     assertEquals(KAFKA_2_1_IV0, ApiVersion("2.1-IV0"))
+    assertEquals(KAFKA_2_1_IV1, ApiVersion("2.1-IV1"))
   }
 
   @Test
diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index c456433..8c1d95a 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -18,6 +18,7 @@
 package kafka.server
 
 import java.nio.ByteBuffer
+import java.util.Optional
 
 import AbstractFetcherThread._
 import com.yammer.metrics.Metrics
@@ -371,7 +372,8 @@ class AbstractFetcherThreadTest {
       partitionMap.foreach { case (partition, state) =>
         if (state.isReadyForFetch) {
           val replicaState = replicaPartitionState(partition)
-          fetchData.put(partition, new FetchRequest.PartitionData(state.fetchOffset, replicaState.logStartOffset, 1024 * 1024))
+          fetchData.put(partition, new FetchRequest.PartitionData(state.fetchOffset, replicaState.logStartOffset,
+            1024 * 1024, Optional.empty()))
         }
       }
       val fetchRequest = FetchRequest.Builder.forReplica(ApiKeys.FETCH.latestVersion, replicaId, 0, 1, fetchData.asJava)
diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala
index 1bf6f28..673abe6 100644
--- a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala
@@ -17,7 +17,7 @@
 package kafka.server
 
 import java.util
-import java.util.Properties
+import java.util.{Optional, Properties}
 
 import kafka.log.LogConfig
 import kafka.utils.TestUtils
@@ -72,7 +72,8 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
                                  offsetMap: Map[TopicPartition, Long] = Map.empty): util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] = {
     val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
     topicPartitions.foreach { tp =>
-      partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), 0L, maxPartitionBytes))
+      partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), 0L,
+        maxPartitionBytes, Optional.empty()))
     }
     partitionMap
   }
diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
index 694b19d..f8c02cf 100644
--- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
@@ -18,7 +18,7 @@ package kafka.server
 
 import java.io.DataInputStream
 import java.util
-import java.util.Properties
+import java.util.{Optional, Properties}
 
 import kafka.api.KAFKA_0_11_0_IV2
 import kafka.log.LogConfig
@@ -58,7 +58,8 @@ class FetchRequestTest extends BaseRequestTest {
                                  offsetMap: Map[TopicPartition, Long] = Map.empty): util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] = {
     val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
     topicPartitions.foreach { tp =>
-      partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), 0L, maxPartitionBytes))
+      partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), 0L, maxPartitionBytes,
+        Optional.empty()))
     }
     partitionMap
   }
diff --git a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
index c4a9625..74dae7c 100755
--- a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
@@ -17,7 +17,7 @@
 package kafka.server
 
 import java.util
-import java.util.Collections
+import java.util.{Collections, Optional}
 
 import kafka.utils.MockTime
 import org.apache.kafka.common.TopicPartition
@@ -141,8 +141,10 @@ class FetchSessionTest {
 
     // Create a new fetch session with a FULL fetch request
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData2.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100))
-    reqData2.put(new TopicPartition("foo", 1), new FetchRequest.PartitionData(10, 0, 100))
+    reqData2.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+      Optional.empty()))
+    reqData2.put(new TopicPartition("foo", 1), new FetchRequest.PartitionData(10, 0, 100,
+      Optional.empty()))
     val context2 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData2, EMPTY_PART_LIST, false)
     assertEquals(classOf[FullFetchContext], context2.getClass)
     val reqData2Iter = reqData2.entrySet().iterator()
@@ -215,8 +217,10 @@ class FetchSessionTest {
     var nextSessionId = prevSessionId
     do {
       val reqData8 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-      reqData8.put(new TopicPartition("bar", 0), new FetchRequest.PartitionData(0, 0, 100))
-      reqData8.put(new TopicPartition("bar", 1), new FetchRequest.PartitionData(10, 0, 100))
+      reqData8.put(new TopicPartition("bar", 0), new FetchRequest.PartitionData(0, 0, 100,
+        Optional.empty()))
+      reqData8.put(new TopicPartition("bar", 1), new FetchRequest.PartitionData(10, 0, 100,
+        Optional.empty()))
       val context8 = fetchManager.newContext(
         new JFetchMetadata(prevSessionId, FINAL_EPOCH), reqData8, EMPTY_PART_LIST, false)
       assertEquals(classOf[SessionlessFetchContext], context8.getClass)
@@ -240,8 +244,10 @@ class FetchSessionTest {
 
     // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100))
-    reqData1.put(new TopicPartition("foo", 1), new FetchRequest.PartitionData(10, 0, 100))
+    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+      Optional.empty()))
+    reqData1.put(new TopicPartition("foo", 1), new FetchRequest.PartitionData(10, 0, 100,
+      Optional.empty()))
     val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
     assertEquals(classOf[FullFetchContext], context1.getClass)
     val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]]
@@ -256,7 +262,8 @@ class FetchSessionTest {
 
     // Create an incremental fetch request that removes foo-0 and adds bar-0
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData2.put(new TopicPartition("bar", 0), new FetchRequest.PartitionData(15, 0, 0))
+    reqData2.put(new TopicPartition("bar", 0), new FetchRequest.PartitionData(15, 0, 0,
+      Optional.empty()))
     val removed2 = new util.ArrayList[TopicPartition]
     removed2.add(new TopicPartition("foo", 0))
     val context2 = fetchManager.newContext(
@@ -290,8 +297,10 @@ class FetchSessionTest {
 
     // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100))
-    reqData1.put(new TopicPartition("foo", 1), new FetchRequest.PartitionData(10, 0, 100))
+    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+      Optional.empty()))
+    reqData1.put(new TopicPartition("foo", 1), new FetchRequest.PartitionData(10, 0, 100,
+      Optional.empty()))
     val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
     assertEquals(classOf[FullFetchContext], context1.getClass)
     val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]]
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index 18b18fd..eb3f28e 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -20,7 +20,7 @@ package kafka.server
 import java.lang.{Long => JLong}
 import java.net.InetAddress
 import java.util
-import java.util.Collections
+import java.util.{Collections, Optional}
 
 import kafka.api.{ApiVersion, KAFKA_0_10_2_IV0}
 import kafka.cluster.Replica
@@ -118,7 +118,7 @@ class KafkaApisTest {
       EasyMock.reset(replicaManager, clientRequestQuotaManager, requestChannel)
 
       val invalidTopicPartition = new TopicPartition(topic, invalidPartitionId)
-      val partitionOffsetCommitData = new OffsetCommitRequest.PartitionData(15L, "")
+      val partitionOffsetCommitData = new OffsetCommitRequest.PartitionData(15L, 23, "")
       val (offsetCommitRequest, request) = buildRequest(new OffsetCommitRequest.Builder("groupId",
         Map(invalidTopicPartition -> partitionOffsetCommitData).asJava))
 
@@ -144,7 +144,7 @@ class KafkaApisTest {
       EasyMock.reset(replicaManager, clientRequestQuotaManager, requestChannel)
 
       val invalidTopicPartition = new TopicPartition(topic, invalidPartitionId)
-      val partitionOffsetCommitData = new TxnOffsetCommitRequest.CommittedOffset(15L, "")
+      val partitionOffsetCommitData = new TxnOffsetCommitRequest.CommittedOffset(15L, "", Optional.empty())
       val (offsetCommitRequest, request) = buildRequest(new TxnOffsetCommitRequest.Builder("txnlId", "groupId",
         15L, 0.toShort, Map(invalidTopicPartition -> partitionOffsetCommitData).asJava))
 
@@ -377,8 +377,10 @@ class KafkaApisTest {
     val capturedResponse = expectNoThrottling()
     EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, replica, log)
 
+
+    val targetTimes = Map(tp -> new ListOffsetRequest.PartitionData(timestamp, Optional.empty[Integer]()))
     val builder = ListOffsetRequest.Builder.forConsumer(true, isolationLevel)
-      .setTargetTimes(Map(tp -> timestamp).asJava)
+      .setTargetTimes(targetTimes.asJava)
     val (listOffsetRequest, request) = buildRequest(builder)
     createKafkaApis().handleListOffsetRequest(request)
 
@@ -418,8 +420,10 @@ class KafkaApisTest {
     val capturedResponse = expectNoThrottling()
     EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, replica, log)
 
+    val targetTimes = Map(tp -> new ListOffsetRequest.PartitionData(ListOffsetRequest.EARLIEST_TIMESTAMP,
+      Optional.empty[Integer]()))
     val builder = ListOffsetRequest.Builder.forConsumer(true, isolationLevel)
-      .setTargetTimes(Map(tp -> (ListOffsetRequest.EARLIEST_TIMESTAMP: JLong)).asJava)
+      .setTargetTimes(targetTimes.asJava)
     val (listOffsetRequest, request) = buildRequest(builder)
     createKafkaApis().handleListOffsetRequest(request)
 
@@ -508,8 +512,10 @@ class KafkaApisTest {
 
     EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, replica, log)
 
+    val targetTimes = Map(tp -> new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP,
+      Optional.empty[Integer]()))
     val builder = ListOffsetRequest.Builder.forConsumer(true, isolationLevel)
-      .setTargetTimes(Map(tp -> (ListOffsetRequest.LATEST_TIMESTAMP: JLong)).asJava)
+      .setTargetTimes(targetTimes.asJava)
     val (listOffsetRequest, request) = buildRequest(builder)
     createKafkaApis().handleListOffsetRequest(request)
 
diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala
index 6ee47ee..965413e 100644
--- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala
@@ -16,7 +16,7 @@
  */
 package kafka.server
 
-import java.lang.{Long => JLong}
+import java.util.Optional
 
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
@@ -33,20 +33,22 @@ class ListOffsetsRequestTest extends BaseRequestTest {
   def testListOffsetsErrorCodes(): Unit = {
     val topic = "topic"
     val partition = new TopicPartition(topic, 0)
+    val targetTimes = Map(partition -> new ListOffsetRequest.PartitionData(
+      ListOffsetRequest.EARLIEST_TIMESTAMP, Optional.of[Integer](0))).asJava
 
     val consumerRequest = ListOffsetRequest.Builder
       .forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
-      .setTargetTimes(Map(partition -> ListOffsetRequest.EARLIEST_TIMESTAMP.asInstanceOf[JLong]).asJava)
+      .setTargetTimes(targetTimes)
       .build()
 
     val replicaRequest = ListOffsetRequest.Builder
       .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, servers.head.config.brokerId)
-      .setTargetTimes(Map(partition -> ListOffsetRequest.EARLIEST_TIMESTAMP.asInstanceOf[JLong]).asJava)
+      .setTargetTimes(targetTimes)
       .build()
 
     val debugReplicaRequest = ListOffsetRequest.Builder
       .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, ListOffsetRequest.DEBUGGING_REPLICA_ID)
-      .setTargetTimes(Map(partition -> ListOffsetRequest.EARLIEST_TIMESTAMP.asInstanceOf[JLong]).asJava)
+      .setTargetTimes(targetTimes)
       .build()
 
     // Unknown topic
diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
index e371f7f..45096cc 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -19,7 +19,7 @@ package kafka.server
 
 import java.io.File
 import java.util.concurrent.atomic.AtomicInteger
-import java.util.{Properties, Random}
+import java.util.{Optional, Properties, Random}
 
 import kafka.log.{Log, LogSegment}
 import kafka.network.SocketServer
@@ -54,7 +54,7 @@ class LogOffsetTest extends BaseRequestTest {
   def testGetOffsetsForUnknownTopic() {
     val topicPartition = new TopicPartition("foo", 0)
     val request = ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
-      .setOffsetData(Map(topicPartition ->
+      .setTargetTimes(Map(topicPartition ->
         new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP, 10)).asJava).build(0)
     val response = sendListOffsetsRequest(request)
     assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response.responseData.get(topicPartition).error)
@@ -86,7 +86,7 @@ class LogOffsetTest extends BaseRequestTest {
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
       "Leader should be elected")
     val request = ListOffsetRequest.Builder.forReplica(0, 0)
-      .setOffsetData(Map(topicPartition ->
+      .setTargetTimes(Map(topicPartition ->
         new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP, 15)).asJava).build()
     val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala
     assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 3L), consumerOffsets)
@@ -114,7 +114,7 @@ class LogOffsetTest extends BaseRequestTest {
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
       "Leader should be elected")
     val request = ListOffsetRequest.Builder.forReplica(0, 0)
-      .setOffsetData(Map(topicPartition ->
+      .setTargetTimes(Map(topicPartition ->
         new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP, 15)).asJava).build()
     val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala
     assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets)
@@ -122,7 +122,7 @@ class LogOffsetTest extends BaseRequestTest {
     // try to fetch using latest offset
     val fetchRequest = FetchRequest.Builder.forConsumer(0, 1,
       Map(topicPartition -> new FetchRequest.PartitionData(consumerOffsets.head, FetchRequest.INVALID_LOG_START_OFFSET,
-        300 * 1024)).asJava).build()
+        300 * 1024, Optional.empty())).asJava).build()
     val fetchResponse = sendFetchRequest(fetchRequest)
     assertFalse(fetchResponse.responseData.get(topicPartition).records.batches.iterator.hasNext)
   }
@@ -142,7 +142,7 @@ class LogOffsetTest extends BaseRequestTest {
     for (_ <- 1 to 14) {
       val topicPartition = new TopicPartition(topic, 0)
       val request = ListOffsetRequest.Builder.forReplica(0, 0)
-        .setOffsetData(Map(topicPartition ->
+        .setTargetTimes(Map(topicPartition ->
           new ListOffsetRequest.PartitionData(ListOffsetRequest.EARLIEST_TIMESTAMP, 1)).asJava).build()
       val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala
       if (consumerOffsets.head == 1)
@@ -174,7 +174,7 @@ class LogOffsetTest extends BaseRequestTest {
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
       "Leader should be elected")
     val request = ListOffsetRequest.Builder.forReplica(0, 0)
-      .setOffsetData(Map(topicPartition ->
+      .setTargetTimes(Map(topicPartition ->
         new ListOffsetRequest.PartitionData(now, 15)).asJava).build()
     val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala
     assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets)
@@ -201,7 +201,7 @@ class LogOffsetTest extends BaseRequestTest {
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
       "Leader should be elected")
     val request = ListOffsetRequest.Builder.forReplica(0, 0)
-      .setOffsetData(Map(topicPartition ->
+      .setTargetTimes(Map(topicPartition ->
         new ListOffsetRequest.PartitionData(ListOffsetRequest.EARLIEST_TIMESTAMP, 10)).asJava).build()
     val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala
     assertEquals(Seq(0L), consumerOffsets)
diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
index 93ac62d..ec3534f 100644
--- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
+++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
@@ -17,6 +17,7 @@
 package kafka.server
 
 import java.util
+import java.util.Optional
 import util.Arrays.asList
 
 import org.apache.kafka.common.TopicPartition
@@ -45,7 +46,6 @@ class MetadataCacheTest {
     val topic0 = "topic-0"
     val topic1 = "topic-1"
 
-
     val cache = new MetadataCache(1)
 
     val zkVersion = 3
@@ -95,6 +95,7 @@ class MetadataCacheTest {
           val leader = partitionMetadata.leader
           val partitionState = topicPartitionStates(new TopicPartition(topic, partitionId))
           assertEquals(partitionState.basePartitionState.leader, leader.id)
+          assertEquals(Optional.of(partitionState.basePartitionState.leaderEpoch), partitionMetadata.leaderEpoch)
           assertEquals(partitionState.basePartitionState.isr, partitionMetadata.isr.asScala.map(_.id).asJava)
           assertEquals(partitionState.basePartitionState.replicas, partitionMetadata.replicas.asScala.map(_.id).asJava)
           val endPoint = endPoints(partitionMetadata.leader.id).find(_.listenerName == listenerName).get
diff --git a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala
index c6385f3..e314b44 100644
--- a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala
@@ -16,6 +16,8 @@
  */
 package kafka.server
 
+import java.util.Optional
+
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
@@ -33,7 +35,7 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
     val partition = new TopicPartition(topic, 0)
 
     val request = new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion)
-      .add(partition, 0)
+      .add(partition, Optional.of(5), 0)
       .build()
 
     // Unknown topic
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
index 9c759ce..520801c 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
@@ -486,7 +486,7 @@ class ReplicaFetcherThreadTest {
     assertEquals(2, mockNetwork.epochFetchCount)
     assertEquals(1, mockNetwork.fetchCount)
     assertEquals("OffsetsForLeaderEpochRequest version.",
-                 1, mockNetwork.lastUsedOffsetForLeaderEpochVersion)
+      2, mockNetwork.lastUsedOffsetForLeaderEpochVersion)
 
     //Loop 3 we should not fetch epochs
     thread.doWork()
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index 66a2c8e..ea0a8ef 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -17,7 +17,7 @@
 package kafka.server
 
 import java.io.File
-import java.util.Properties
+import java.util.{Optional, Properties}
 import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.cluster.Replica
@@ -43,7 +43,9 @@ class ReplicaManagerQuotasTest {
   val record = new SimpleRecord("some-data-in-a-message".getBytes())
   val topicPartition1 = new TopicPartition("test-topic", 1)
   val topicPartition2 = new TopicPartition("test-topic", 2)
-  val fetchInfo = Seq(topicPartition1 -> new PartitionData(0, 0, 100), topicPartition2 -> new PartitionData(0, 0, 100))
+  val fetchInfo = Seq(
+    topicPartition1 -> new PartitionData(0, 0, 100, Optional.empty()),
+    topicPartition2 -> new PartitionData(0, 0, 100, Optional.empty()))
   var replicaManager: ReplicaManager = _
 
   @Test
@@ -164,9 +166,9 @@ class ReplicaManagerQuotasTest {
       EasyMock.replay(replicaManager)
 
       val tp = new TopicPartition("t1", 0)
-      val fetchParititonStatus = new FetchPartitionStatus(new LogOffsetMetadata(messageOffset = 50L, segmentBaseOffset = 0L,
-        relativePositionInSegment = 250), new PartitionData(50, 0, 1))
-      val fetchMetadata = new FetchMetadata(fetchMinBytes = 1, fetchMaxBytes = 1000, hardMaxBytesLimit = true, fetchOnlyLeader = true,
+      val fetchParititonStatus = FetchPartitionStatus(new LogOffsetMetadata(messageOffset = 50L, segmentBaseOffset = 0L,
+         relativePositionInSegment = 250), new PartitionData(50, 0, 1, Optional.empty()))
+      val fetchMetadata = FetchMetadata(fetchMinBytes = 1, fetchMaxBytes = 1000, hardMaxBytesLimit = true, fetchOnlyLeader = true,
         fetchOnlyCommitted = false, isFromFollower = true, replicaId = 1, fetchPartitionStatus = List((tp, fetchParititonStatus)))
       new DelayedFetch(delayMs = 600, fetchMetadata = fetchMetadata, replicaManager = replicaManager,
         quota = null, isolationLevel = IsolationLevel.READ_UNCOMMITTED, responseCallback = null) {
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 41564a5..0844052 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -18,7 +18,7 @@
 package kafka.server
 
 import java.io.File
-import java.util.Properties
+import java.util.{Optional, Properties}
 import java.util.concurrent.{CountDownLatch, TimeUnit}
 import java.util.concurrent.atomic.AtomicBoolean
 
@@ -162,7 +162,8 @@ class ReplicaManagerTest {
       partition.getOrCreateReplica(0)
       // Make this replica the leader.
       val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
+        collection.immutable.Map(new TopicPartition(topic, 0) ->
+          new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -173,13 +174,15 @@ class ReplicaManagerTest {
       }
 
       // Fetch some messages
-      val fetchResult = fetchAsConsumer(rm, new TopicPartition(topic, 0), new PartitionData(0, 0, 100000),
+      val fetchResult = fetchAsConsumer(rm, new TopicPartition(topic, 0),
+        new PartitionData(0, 0, 100000, Optional.empty()),
         minBytes = 100000)
       assertFalse(fetchResult.isFired)
 
       // Make this replica the follower
       val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new LeaderAndIsrRequest.PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava,
+        collection.immutable.Map(new TopicPartition(topic, 0) ->
+          new LeaderAndIsrRequest.PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ())
 
@@ -203,7 +206,8 @@ class ReplicaManagerTest {
 
       // Make this replica the leader.
       val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
+        collection.immutable.Map(new TopicPartition(topic, 0) ->
+          new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -252,7 +256,8 @@ class ReplicaManagerTest {
 
       // Make this replica the leader.
       val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
+        collection.immutable.Map(new TopicPartition(topic, 0) ->
+          new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -272,12 +277,14 @@ class ReplicaManagerTest {
       }
 
       // fetch as follower to advance the high watermark
-      fetchAsFollower(replicaManager, new TopicPartition(topic, 0), new PartitionData(numRecords, 0, 100000),
+      fetchAsFollower(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(numRecords, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_UNCOMMITTED)
 
       // fetch should return empty since LSO should be stuck at 0
       var consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0),
-        new PartitionData(0, 0, 100000), isolationLevel = IsolationLevel.READ_COMMITTED)
+        new PartitionData(0, 0, 100000, Optional.empty()),
+        isolationLevel = IsolationLevel.READ_COMMITTED)
       var fetchData = consumerFetchResult.assertFired
       assertEquals(Errors.NONE, fetchData.error)
       assertTrue(fetchData.records.batches.asScala.isEmpty)
@@ -285,7 +292,8 @@ class ReplicaManagerTest {
       assertEquals(Some(List.empty[AbortedTransaction]), fetchData.abortedTransactions)
 
       // delayed fetch should timeout and return nothing
-      consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0), new PartitionData(0, 0, 100000),
+      consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(0, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_COMMITTED, minBytes = 1000)
       assertFalse(consumerFetchResult.isFired)
       timer.advanceClock(1001)
@@ -304,7 +312,8 @@ class ReplicaManagerTest {
 
       // the LSO has advanced, but the appended commit marker has not been replicated, so
       // none of the data from the transaction should be visible yet
-      consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0), new PartitionData(0, 0, 100000),
+      consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(0, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_COMMITTED)
 
       fetchData = consumerFetchResult.assertFired
@@ -312,11 +321,13 @@ class ReplicaManagerTest {
       assertTrue(fetchData.records.batches.asScala.isEmpty)
 
       // fetch as follower to advance the high watermark
-      fetchAsFollower(replicaManager, new TopicPartition(topic, 0), new PartitionData(numRecords + 1, 0, 100000),
+      fetchAsFollower(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(numRecords + 1, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_UNCOMMITTED)
 
       // now all of the records should be fetchable
-      consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0), new PartitionData(0, 0, 100000),
+      consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(0, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_COMMITTED)
 
       fetchData = consumerFetchResult.assertFired
@@ -341,7 +352,8 @@ class ReplicaManagerTest {
 
       // Make this replica the leader.
       val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
+        collection.immutable.Map(new TopicPartition(topic, 0) ->
+          new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -366,12 +378,14 @@ class ReplicaManagerTest {
         .onFire { response => assertEquals(Errors.NONE, response.error) }
 
       // fetch as follower to advance the high watermark
-      fetchAsFollower(replicaManager, new TopicPartition(topic, 0), new PartitionData(numRecords + 1, 0, 100000),
+      fetchAsFollower(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(numRecords + 1, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_UNCOMMITTED)
 
       // Set the minBytes in order force this request to enter purgatory. When it returns, we should still
       // see the newly aborted transaction.
-      val fetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0), new PartitionData(0, 0, 100000),
+      val fetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0),
+        new PartitionData(0, 0, 100000, Optional.empty()),
         isolationLevel = IsolationLevel.READ_COMMITTED, minBytes = 10000)
       assertFalse(fetchResult.isFired)
 
@@ -403,7 +417,8 @@ class ReplicaManagerTest {
 
       // Make this replica the leader.
       val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
+        collection.immutable.Map(new TopicPartition(topic, 0) ->
+          new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava).build()
       rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -417,13 +432,15 @@ class ReplicaManagerTest {
       }
 
       // Fetch a message above the high watermark as a follower
-      val followerFetchResult = fetchAsFollower(rm, new TopicPartition(topic, 0), new PartitionData(1, 0, 100000))
+      val followerFetchResult = fetchAsFollower(rm, new TopicPartition(topic, 0),
+        new PartitionData(1, 0, 100000, Optional.empty()))
       val followerFetchData = followerFetchResult.assertFired
       assertEquals("Should not give an exception", Errors.NONE, followerFetchData.error)
       assertTrue("Should return some data", followerFetchData.records.batches.iterator.hasNext)
 
       // Fetch a message above the high watermark as a consumer
-      val consumerFetchResult = fetchAsConsumer(rm, new TopicPartition(topic, 0), new PartitionData(1, 0, 100000))
+      val consumerFetchResult = fetchAsConsumer(rm, new TopicPartition(topic, 0),
+        new PartitionData(1, 0, 100000, Optional.empty()))
       val consumerFetchData = consumerFetchResult.assertFired
       assertEquals("Should not give an exception", Errors.NONE, consumerFetchData.error)
       assertEquals("Should return empty response", MemoryRecords.EMPTY, consumerFetchData.records)
@@ -495,8 +512,8 @@ class ReplicaManagerTest {
         fetchMaxBytes = Int.MaxValue,
         hardMaxBytesLimit = false,
         fetchInfos = Seq(
-          tp0 -> new PartitionData(1, 0, 100000),
-          tp1 -> new PartitionData(1, 0, 100000)),
+          tp0 -> new PartitionData(1, 0, 100000, Optional.empty()),
+          tp1 -> new PartitionData(1, 0, 100000, Optional.empty())),
         responseCallback = fetchCallback,
         isolationLevel = IsolationLevel.READ_UNCOMMITTED
       )
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index d91e008..3b7ecfb 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -15,7 +15,7 @@
 package kafka.server
 
 import java.nio.ByteBuffer
-import java.util.{Collections, LinkedHashMap, Properties}
+import java.util.{Collections, LinkedHashMap, Optional, Properties}
 import java.util.concurrent.{Executors, Future, TimeUnit}
 
 import kafka.log.LogConfig
@@ -207,7 +207,7 @@ class RequestQuotaTest extends BaseRequestTest {
 
         case ApiKeys.FETCH =>
           val partitionMap = new LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-          partitionMap.put(tp, new FetchRequest.PartitionData(0, 0, 100))
+          partitionMap.put(tp, new FetchRequest.PartitionData(0, 0, 100, Optional.of(15)))
           FetchRequest.Builder.forConsumer(0, 0, partitionMap)
 
         case ApiKeys.METADATA =>
@@ -215,11 +215,13 @@ class RequestQuotaTest extends BaseRequestTest {
 
         case ApiKeys.LIST_OFFSETS =>
           ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
-            .setTargetTimes(Map(tp -> (0L: java.lang.Long)).asJava)
+            .setTargetTimes(Map(tp -> new ListOffsetRequest.PartitionData(
+              0L, Optional.of[Integer](15))).asJava)
 
         case ApiKeys.LEADER_AND_ISR =>
           new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue,
-            Map(tp -> new LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, true)).asJava,
+            Map(tp -> new LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava,
+              2, Seq(brokerId).asJava, true)).asJava,
             Set(new Node(brokerId, "localhost", 0)).asJava)
 
         case ApiKeys.STOP_REPLICA =>
@@ -239,7 +241,7 @@ class RequestQuotaTest extends BaseRequestTest {
 
         case ApiKeys.OFFSET_COMMIT =>
           new OffsetCommitRequest.Builder("test-group",
-            Map(tp -> new OffsetCommitRequest.PartitionData(0, "metadata")).asJava).
+            Map(tp -> new OffsetCommitRequest.PartitionData(0, 15, "metadata")).asJava).
             setMemberId("").setGenerationId(1)
 
         case ApiKeys.OFFSET_FETCH =>
@@ -290,7 +292,8 @@ class RequestQuotaTest extends BaseRequestTest {
           new InitProducerIdRequest.Builder("abc")
 
         case ApiKeys.OFFSET_FOR_LEADER_EPOCH =>
-          new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion()).add(tp, 0)
+          new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion)
+            .add(tp, Optional.of(15), 0)
 
         case ApiKeys.ADD_PARTITIONS_TO_TXN =>
           new AddPartitionsToTxnRequest.Builder("test-transactional-id", 1, 0, List(tp).asJava)
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index 0797b7b..d2f1c0a 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -27,7 +27,7 @@ import kafka.zk.KafkaZkClient
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
 import org.junit.{After, Before, Test}
-import java.util.Properties
+import java.util.{Optional, Properties}
 import java.util.concurrent.atomic.AtomicBoolean
 
 import org.apache.kafka.common.TopicPartition
@@ -63,7 +63,8 @@ class SimpleFetchTest {
   val partitionId = 0
   val topicPartition = new TopicPartition(topic, partitionId)
 
-  val fetchInfo = Seq(topicPartition -> new PartitionData(0, 0, fetchSize))
+  val fetchInfo = Seq(topicPartition -> new PartitionData(0, 0, fetchSize,
+    Optional.empty()))
 
   var replicaManager: ReplicaManager = _
 
diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
index 17683f4..5ad641f 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
@@ -16,7 +16,7 @@
   */
 package kafka.server.epoch
 
-import java.util.{Map => JMap}
+import java.util.{Optional, Map => JMap}
 
 import kafka.server.KafkaConfig._
 import kafka.server.{BlockingSend, KafkaServer, ReplicaFetcherBlockingSend}
@@ -31,7 +31,6 @@ import org.apache.kafka.common.serialization.StringSerializer
 import org.apache.kafka.common.utils.{LogContext, SystemTime}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.ApiKeys
-
 import org.junit.Assert._
 import org.junit.{After, Test}
 import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
@@ -266,13 +265,13 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging {
   private[epoch] class TestFetcherThread(sender: BlockingSend) extends Logging {
 
     def leaderOffsetsFor(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = {
-      val request = new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(), toJavaFormat(partitions))
+      val partitionData = partitions.mapValues(
+        new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), _))
+      val request = new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion,
+        partitionData.asJava)
       val response = sender.sendRequest(request)
       response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse].responses.asScala
     }
 
-    def toJavaFormat(partitions: Map[TopicPartition, Int]): JMap[TopicPartition, Integer] = {
-      partitions.map { case (tp, epoch) => tp -> epoch.asInstanceOf[Integer] }.toMap.asJava
-    }
   }
 }
diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
index 5c60c00..4fdc4d2 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
@@ -17,6 +17,7 @@
 package kafka.server.epoch
 
 import java.io.File
+import java.util.Optional
 import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.cluster.Replica
@@ -25,13 +26,12 @@ import kafka.utils.{MockTime, TestUtils}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.EpochEndOffset
+import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest}
 import org.apache.kafka.common.requests.EpochEndOffset._
 import org.easymock.EasyMock._
 import org.junit.Assert._
 import org.junit.Test
 
-
 class OffsetsForLeaderEpochTest {
   private val config = TestUtils.createBrokerConfigs(1, TestUtils.MockZkConnect).map(KafkaConfig.fromProps).head
   private val time = new MockTime
@@ -43,7 +43,7 @@ class OffsetsForLeaderEpochTest {
     //Given
     val epochAndOffset = (5, 42L)
     val epochRequested: Integer = 5
-    val request = Map(tp -> epochRequested)
+    val request = Map(tp -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), epochRequested))
 
     //Stubs
     val mockLog = createNiceMock(classOf[kafka.log.Log])
@@ -84,7 +84,7 @@ class OffsetsForLeaderEpochTest {
 
     //Given
     val epochRequested: Integer = 5
-    val request = Map(tp -> epochRequested)
+    val request = Map(tp -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), epochRequested))
 
     //When
     val response = replicaManager.lastOffsetForLeaderEpoch(request)
@@ -106,7 +106,7 @@ class OffsetsForLeaderEpochTest {
 
     //Given
     val epochRequested: Integer = 5
-    val request = Map(tp -> epochRequested)
+    val request = Map(tp -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), epochRequested))
 
     //When
     val response = replicaManager.lastOffsetForLeaderEpoch(request)