You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jg...@apache.org on 2018/10/05 20:42:38 UTC

[kafka] branch 2.1 updated (ab6269e -> c226bda)

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

jgus pushed a change to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git.


    from ab6269e  KAFKA-6914; Set parent classloader of DelegatingClassLoader same as the worker's (#5720)
     new bcef30d  KAFKA-7395; Add fencing to replication protocol (KIP-320) (#5661)
     new c226bda  HOTFIX: Compilation error in GroupMetadataManagerTest (#5752)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../kafka/clients/consumer/internals/Fetcher.java  |   9 +-
 ...eption.java => FencedLeaderEpochException.java} |  17 +-
 ...ption.java => UnknownLeaderEpochException.java} |  12 +-
 .../org/apache/kafka/common/protocol/Errors.java   |  13 +-
 .../kafka/common/requests/FetchResponse.java       |  15 +-
 .../kafka/common/requests/ListOffsetResponse.java  |  16 +-
 .../requests/OffsetsForLeaderEpochRequest.java     |  11 +-
 .../requests/OffsetsForLeaderEpochResponse.java    |  13 +
 .../main/scala/kafka/cluster/BrokerEndPoint.scala  |   4 +
 core/src/main/scala/kafka/cluster/Partition.scala  | 360 +++++++++++++++----
 core/src/main/scala/kafka/cluster/Replica.scala    |  10 +-
 .../coordinator/group/GroupMetadataManager.scala   |   8 +-
 .../transaction/TransactionStateManager.scala      |   2 +-
 core/src/main/scala/kafka/log/Log.scala            | 102 ++++--
 .../kafka/server/AbstractFetcherManager.scala      |  55 +--
 .../scala/kafka/server/AbstractFetcherThread.scala | 255 ++++++++-----
 .../scala/kafka/server/DelayedDeleteRecords.scala  |   2 +-
 .../src/main/scala/kafka/server/DelayedFetch.scala |  56 +--
 .../main/scala/kafka/server/FetchDataInfo.scala    |   5 +
 core/src/main/scala/kafka/server/KafkaApis.scala   | 143 ++------
 .../kafka/server/ReplicaAlterLogDirsManager.scala  |  12 +-
 .../kafka/server/ReplicaAlterLogDirsThread.scala   |  51 ++-
 .../scala/kafka/server/ReplicaFetcherManager.scala |  21 +-
 .../scala/kafka/server/ReplicaFetcherThread.scala  |  81 +++--
 .../main/scala/kafka/server/ReplicaManager.scala   | 335 +++++++++--------
 core/src/main/scala/kafka/utils/DelayedItem.scala  |   2 +-
 .../kafka/api/AdminClientIntegrationTest.scala     |  18 +-
 .../kafka/api/AuthorizerIntegrationTest.scala      |   4 +-
 .../integration/kafka/api/ConsumerBounceTest.scala |   2 +-
 .../kafka/server/DelayedFetchTest.scala            | 121 +++++++
 .../server/DynamicBrokerReconfigurationTest.scala  |   6 +-
 .../src/test/scala/other/kafka/StressTestLog.scala |   6 +-
 .../admin/ReassignPartitionsClusterTest.scala      |   7 +-
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 398 ++++++++++++++++++--
 .../group/GroupMetadataManagerTest.scala           |  16 +-
 .../TransactionCoordinatorConcurrencyTest.scala    |   7 +-
 .../transaction/TransactionStateManagerTest.scala  |   7 +-
 .../kafka/integration/KafkaServerTestHarness.scala |   2 +-
 .../unit/kafka/log/BrokerCompressionTest.scala     |   8 +-
 .../test/scala/unit/kafka/log/LogManagerTest.scala |  14 +-
 core/src/test/scala/unit/kafka/log/LogTest.scala   |  56 +--
 .../kafka/server/AbstractFetcherManagerTest.scala  |  61 ++++
 .../kafka/server/AbstractFetcherThreadTest.scala   | 399 +++++++++++++++++----
 .../server/AlterReplicaLogDirsRequestTest.scala    |   2 +-
 .../scala/unit/kafka/server/FetchRequestTest.scala |  36 ++
 .../server/HighwatermarkPersistenceTest.scala      |   8 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala    | 100 ++----
 .../unit/kafka/server/ListOffsetsRequestTest.scala |  40 ++-
 .../unit/kafka/server/LogDirFailureTest.scala      |   4 +-
 .../scala/unit/kafka/server/LogOffsetTest.scala    |  12 +-
 .../scala/unit/kafka/server/LogRecoveryTest.scala  |  12 +-
 .../server/OffsetsForLeaderEpochRequestTest.scala  |  44 ++-
 .../server/ReplicaAlterLogDirsThreadTest.scala     | 270 ++++++++------
 .../kafka/server/ReplicaFetcherThreadTest.scala    |  86 +++--
 .../kafka/server/ReplicaManagerQuotasTest.scala    |  77 ++--
 .../unit/kafka/server/ReplicaManagerTest.scala     |  82 +----
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   4 +-
 .../scala/unit/kafka/server/SimpleFetchTest.scala  |  14 +-
 .../server/epoch/LeaderEpochIntegrationTest.scala  |   3 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala    |  58 ++-
 60 files changed, 2448 insertions(+), 1146 deletions(-)
 copy clients/src/main/java/org/apache/kafka/common/errors/{InvalidTxnTimeoutException.java => FencedLeaderEpochException.java} (67%)
 copy clients/src/main/java/org/apache/kafka/common/errors/{LeaderNotAvailableException.java => UnknownLeaderEpochException.java} (68%)
 create mode 100644 core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala
 create mode 100644 core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala


[kafka] 02/02: HOTFIX: Compilation error in GroupMetadataManagerTest (#5752)

Posted by jg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c226bda5dfd202724d7f1e898e723967d569484a
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Fri Oct 5 13:38:33 2018 -0700

    HOTFIX: Compilation error in GroupMetadataManagerTest (#5752)
    
    Accidentally broke after merging KAFKA-7395 which had not been updated for #5727.
    
    Reviewers:Matthias J. Sax <ma...@confluent.io>
---
 .../unit/kafka/coordinator/group/GroupMetadataManagerTest.scala  | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)

diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
index e10c3ea..9ab9705 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
@@ -29,7 +29,7 @@ import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Subscriptio
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.record._
-import org.apache.kafka.common.requests.{IsolationLevel, OffsetFetchResponse}
+import org.apache.kafka.common.requests.OffsetFetchResponse
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.easymock.{Capture, EasyMock, IAnswer}
 import org.junit.Assert.{assertEquals, assertFalse, assertNull, assertTrue}
@@ -1799,8 +1799,11 @@ class GroupMetadataManagerTest {
 
     val logMock = EasyMock.mock(classOf[Log])
     EasyMock.expect(logMock.logStartOffset).andReturn(startOffset).anyTimes()
-    EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None),
-      EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED)))
+    EasyMock.expect(logMock.read(EasyMock.eq(startOffset),
+      maxLength = EasyMock.anyInt(),
+      maxOffset = EasyMock.eq(None),
+      minOneMessage = EasyMock.eq(true),
+      includeAbortedTxns = EasyMock.eq(false)))
       .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), mockRecords))
     EasyMock.expect(replicaManager.getLog(groupMetadataTopicPartition)).andStubReturn(Some(logMock))
     EasyMock.expect(replicaManager.getLogEndOffset(groupMetadataTopicPartition)).andStubReturn(Some(18))


[kafka] 01/02: KAFKA-7395; Add fencing to replication protocol (KIP-320) (#5661)

Posted by jg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bcef30d614b2e845b34d1dc24f4a7f8338d10fab
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Fri Oct 5 13:25:08 2018 -0700

    KAFKA-7395; Add fencing to replication protocol (KIP-320) (#5661)
    
    This patch contains the broker-side support for the fencing improvements from KIP-320. This includes the leader epoch validation in the ListOffsets, OffsetsForLeaderEpoch, and Fetch APIs as well as the changes needed in the fetcher threads to maintain and use the current leader epoch. The client changes from KIP-320 will be left for a follow-up.
    
    One notable change worth mentioning is that we now require the read lock in `Partition` in order to read from the log or to query offsets. This is necessary to ensure the safety of the leader epoch validation. Additionally, we forward all leader epoch changes to the replica fetcher thread and go through the truncation phase. This is needed to ensure the fetcher always has the latest epoch and to guarantee that we cannot miss needed truncation if we missed an epoch change.
    
    Reviewers: Jun Rao <ju...@gmail.com>
---
 .../kafka/clients/consumer/internals/Fetcher.java  |   9 +-
 .../common/errors/FencedLeaderEpochException.java  |  35 ++
 .../common/errors/UnknownLeaderEpochException.java |  35 ++
 .../org/apache/kafka/common/protocol/Errors.java   |  13 +-
 .../kafka/common/requests/FetchResponse.java       |  15 +-
 .../kafka/common/requests/ListOffsetResponse.java  |  16 +-
 .../requests/OffsetsForLeaderEpochRequest.java     |  11 +-
 .../requests/OffsetsForLeaderEpochResponse.java    |  13 +
 .../main/scala/kafka/cluster/BrokerEndPoint.scala  |   4 +
 core/src/main/scala/kafka/cluster/Partition.scala  | 360 +++++++++++++++----
 core/src/main/scala/kafka/cluster/Replica.scala    |  10 +-
 .../coordinator/group/GroupMetadataManager.scala   |   8 +-
 .../transaction/TransactionStateManager.scala      |   2 +-
 core/src/main/scala/kafka/log/Log.scala            | 102 ++++--
 .../kafka/server/AbstractFetcherManager.scala      |  55 +--
 .../scala/kafka/server/AbstractFetcherThread.scala | 255 ++++++++-----
 .../scala/kafka/server/DelayedDeleteRecords.scala  |   2 +-
 .../src/main/scala/kafka/server/DelayedFetch.scala |  56 +--
 .../main/scala/kafka/server/FetchDataInfo.scala    |   5 +
 core/src/main/scala/kafka/server/KafkaApis.scala   | 143 ++------
 .../kafka/server/ReplicaAlterLogDirsManager.scala  |  12 +-
 .../kafka/server/ReplicaAlterLogDirsThread.scala   |  51 ++-
 .../scala/kafka/server/ReplicaFetcherManager.scala |  21 +-
 .../scala/kafka/server/ReplicaFetcherThread.scala  |  81 +++--
 .../main/scala/kafka/server/ReplicaManager.scala   | 335 +++++++++--------
 core/src/main/scala/kafka/utils/DelayedItem.scala  |   2 +-
 .../kafka/api/AdminClientIntegrationTest.scala     |  18 +-
 .../kafka/api/AuthorizerIntegrationTest.scala      |   4 +-
 .../integration/kafka/api/ConsumerBounceTest.scala |   2 +-
 .../kafka/server/DelayedFetchTest.scala            | 121 +++++++
 .../server/DynamicBrokerReconfigurationTest.scala  |   6 +-
 .../src/test/scala/other/kafka/StressTestLog.scala |   6 +-
 .../admin/ReassignPartitionsClusterTest.scala      |   7 +-
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 398 ++++++++++++++++++--
 .../group/GroupMetadataManagerTest.scala           |   7 +-
 .../TransactionCoordinatorConcurrencyTest.scala    |   7 +-
 .../transaction/TransactionStateManagerTest.scala  |   7 +-
 .../kafka/integration/KafkaServerTestHarness.scala |   2 +-
 .../unit/kafka/log/BrokerCompressionTest.scala     |   8 +-
 .../test/scala/unit/kafka/log/LogManagerTest.scala |  14 +-
 core/src/test/scala/unit/kafka/log/LogTest.scala   |  56 +--
 .../kafka/server/AbstractFetcherManagerTest.scala  |  61 ++++
 .../kafka/server/AbstractFetcherThreadTest.scala   | 399 +++++++++++++++++----
 .../server/AlterReplicaLogDirsRequestTest.scala    |   2 +-
 .../scala/unit/kafka/server/FetchRequestTest.scala |  36 ++
 .../server/HighwatermarkPersistenceTest.scala      |   8 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala    | 100 ++----
 .../unit/kafka/server/ListOffsetsRequestTest.scala |  40 ++-
 .../unit/kafka/server/LogDirFailureTest.scala      |   4 +-
 .../scala/unit/kafka/server/LogOffsetTest.scala    |  12 +-
 .../scala/unit/kafka/server/LogRecoveryTest.scala  |  12 +-
 .../server/OffsetsForLeaderEpochRequestTest.scala  |  44 ++-
 .../server/ReplicaAlterLogDirsThreadTest.scala     | 270 ++++++++------
 .../kafka/server/ReplicaFetcherThreadTest.scala    |  86 +++--
 .../kafka/server/ReplicaManagerQuotasTest.scala    |  77 ++--
 .../unit/kafka/server/ReplicaManagerTest.scala     |  82 +----
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   4 +-
 .../scala/unit/kafka/server/SimpleFetchTest.scala  |  14 +-
 .../server/epoch/LeaderEpochIntegrationTest.scala  |   3 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala    |  58 ++-
 60 files changed, 2497 insertions(+), 1129 deletions(-)

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 6b42d07..93abc40 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
@@ -808,9 +808,11 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                 // offset corresponding to the requested timestamp and leave it out of the result.
                 log.debug("Cannot search by timestamp for partition {} because the message format version " +
                         "is before 0.10.0", topicPartition);
-            } else if (error == Errors.NOT_LEADER_FOR_PARTITION) {
-                log.debug("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
-                        topicPartition);
+            } else if (error == Errors.NOT_LEADER_FOR_PARTITION ||
+                       error == Errors.REPLICA_NOT_AVAILABLE ||
+                       error == Errors.KAFKA_STORAGE_ERROR) {
+                log.debug("Attempt to fetch offsets for partition {} failed due to {}, retrying.",
+                        topicPartition, error);
                 partitionsToRetry.add(topicPartition);
             } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
                 log.warn("Received unknown topic or partition error in ListOffset request for partition {}", topicPartition);
@@ -967,6 +969,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                     subscriptions.updateLastStableOffset(tp, partition.lastStableOffset);
                 }
             } else if (error == Errors.NOT_LEADER_FOR_PARTITION ||
+                       error == Errors.REPLICA_NOT_AVAILABLE ||
                        error == Errors.KAFKA_STORAGE_ERROR) {
                 log.debug("Error in fetch for partition {}: {}", tp, error.exceptionName());
                 this.metadata.requestUpdate();
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FencedLeaderEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/FencedLeaderEpochException.java
new file mode 100644
index 0000000..24f0eef
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/FencedLeaderEpochException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+/**
+ * The request contained a leader epoch which is smaller than that on the broker that received the
+ * request. This can happen when an operation is attempted before a pending metadata update has been
+ * received. Clients will typically refresh metadata before retrying.
+ */
+public class FencedLeaderEpochException extends InvalidMetadataException {
+    private static final long serialVersionUID = 1L;
+
+    public FencedLeaderEpochException(String message) {
+        super(message);
+    }
+
+    public FencedLeaderEpochException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownLeaderEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownLeaderEpochException.java
new file mode 100644
index 0000000..3714c36
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownLeaderEpochException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+/**
+ * The request contained a leader epoch which is larger than that on the broker that received the
+ * request. This can happen if the client observes a metadata update before it has been propagated
+ * to all brokers. Clients need not refresh metadata before retrying.
+ */
+public class UnknownLeaderEpochException extends RetriableException {
+    private static final long serialVersionUID = 1L;
+
+    public UnknownLeaderEpochException(String message) {
+        super(message);
+    }
+
+    public UnknownLeaderEpochException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 082eff1..7cf5a29 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -30,6 +30,7 @@ import org.apache.kafka.common.errors.DelegationTokenDisabledException;
 import org.apache.kafka.common.errors.DelegationTokenExpiredException;
 import org.apache.kafka.common.errors.DelegationTokenNotFoundException;
 import org.apache.kafka.common.errors.DelegationTokenOwnerMismatchException;
+import org.apache.kafka.common.errors.FencedLeaderEpochException;
 import org.apache.kafka.common.errors.ListenerNotFoundException;
 import org.apache.kafka.common.errors.FetchSessionIdNotFoundException;
 import org.apache.kafka.common.errors.GroupAuthorizationException;
@@ -84,6 +85,7 @@ import org.apache.kafka.common.errors.TopicDeletionDisabledException;
 import org.apache.kafka.common.errors.TopicExistsException;
 import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
 import org.apache.kafka.common.errors.TransactionCoordinatorFencedException;
+import org.apache.kafka.common.errors.UnknownLeaderEpochException;
 import org.apache.kafka.common.errors.UnknownMemberIdException;
 import org.apache.kafka.common.errors.UnknownProducerIdException;
 import org.apache.kafka.common.errors.UnknownServerException;
@@ -278,7 +280,11 @@ public enum Errors {
             "metadata request was processed.",
             ListenerNotFoundException::new),
     TOPIC_DELETION_DISABLED(73, "Topic deletion is disabled.",
-            TopicDeletionDisabledException::new);
+            TopicDeletionDisabledException::new),
+    FENCED_LEADER_EPOCH(74, "The leader epoch in the request is older than the epoch on the broker",
+            FencedLeaderEpochException::new),
+    UNKNOWN_LEADER_EPOCH(75, "The leader epoch in the request is newer than the epoch on the broker",
+            UnknownLeaderEpochException::new);
 
     private static final Logger log = LoggerFactory.getLogger(Errors.class);
 
@@ -287,7 +293,10 @@ public enum Errors {
 
     static {
         for (Errors error : Errors.values()) {
-            codeToError.put(error.code(), error);
+            if (codeToError.put(error.code(), error) != null)
+                throw new ExceptionInInitializerError("Code " + error.code() + " for error " +
+                        error + " has already been used");
+
             if (error.exception != null)
                 classToError.put(error.exception.getClass(), error);
         }
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 2e0eaf2..f87f2ef 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
@@ -52,11 +52,16 @@ import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
  *
  * Possible error codes:
  *
- *  OFFSET_OUT_OF_RANGE (1)
- *  UNKNOWN_TOPIC_OR_PARTITION (3)
- *  NOT_LEADER_FOR_PARTITION (6)
- *  REPLICA_NOT_AVAILABLE (9)
- *  UNKNOWN (-1)
+ * - {@link Errors#OFFSET_OUT_OF_RANGE} If the fetch offset is out of range for a requested partition
+ * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} If the user does not have READ access to a requested topic
+ * - {@link Errors#REPLICA_NOT_AVAILABLE} If the request is received by a broker which is not a replica
+ * - {@link Errors#NOT_LEADER_FOR_PARTITION} If the broker is not a leader and either the provided leader epoch
+ *     matches the known leader epoch on the broker or is empty
+ * - {@link Errors#FENCED_LEADER_EPOCH} If the epoch is lower than the broker's epoch
+ * - {@link Errors#UNKNOWN_LEADER_EPOCH} If the epoch is larger than the broker's epoch
+ * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} If the broker does not have metadata for a topic or partition
+ * - {@link Errors#KAFKA_STORAGE_ERROR} If the log directory for one of the requested partitions is offline
+ * - {@link Errors#UNKNOWN_SERVER_ERROR} For any unexpected errors
  */
 public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
 
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 9f3ce73..188571b 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
@@ -40,12 +40,18 @@ import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 import static org.apache.kafka.common.protocol.types.Type.INT64;
 
 /**
- * Possible error code:
+ * Possible error codes:
  *
- *  UNKNOWN_TOPIC_OR_PARTITION (3)
- *  NOT_LEADER_FOR_PARTITION (6)
- *  UNSUPPORTED_FOR_MESSAGE_FORMAT (43)
- *  UNKNOWN (-1)
+ * - {@link Errors#UNSUPPORTED_FOR_MESSAGE_FORMAT} If the message format does not support lookup by timestamp
+ * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} If the user does not have DESCRIBE access to a requested topic
+ * - {@link Errors#REPLICA_NOT_AVAILABLE} If the request is received by a broker which is not a replica
+ * - {@link Errors#NOT_LEADER_FOR_PARTITION} If the broker is not a leader and either the provided leader epoch
+ *     matches the known leader epoch on the broker or is empty
+ * - {@link Errors#FENCED_LEADER_EPOCH} If the epoch is lower than the broker's epoch
+ * - {@link Errors#UNKNOWN_LEADER_EPOCH} If the epoch is larger than the broker's epoch
+ * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} If the broker does not have metadata for a topic or partition
+ * - {@link Errors#KAFKA_STORAGE_ERROR} If the log directory for one of the requested partitions is offline
+ * - {@link Errors#UNKNOWN_SERVER_ERROR} For any unexpected errors
  */
 public class ListOffsetResponse extends AbstractResponse {
     public static final long UNKNOWN_TIMESTAMP = -1L;
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 1c9009c..feeb875 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
@@ -79,22 +79,13 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
     }
 
     public static class Builder extends AbstractRequest.Builder<OffsetsForLeaderEpochRequest> {
-        private Map<TopicPartition, PartitionData> epochsByPartition;
-
-        public Builder(short version) {
-            this(version, new HashMap<>());
-        }
+        private final Map<TopicPartition, PartitionData> epochsByPartition;
 
         public Builder(short version, Map<TopicPartition, PartitionData> epochsByPartition) {
             super(ApiKeys.OFFSET_FOR_LEADER_EPOCH, version);
             this.epochsByPartition = epochsByPartition;
         }
 
-        public Builder add(TopicPartition topicPartition, Optional<Integer> currentEpoch, int leaderEpoch) {
-            epochsByPartition.put(topicPartition, new PartitionData(currentEpoch, leaderEpoch));
-            return this;
-        }
-
         @Override
         public OffsetsForLeaderEpochRequest build(short version) {
             return new OffsetsForLeaderEpochRequest(epochsByPartition, version);
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 55aa71b..6f70850 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
@@ -37,6 +37,19 @@ 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:
+ *
+ * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} If the user does not have DESCRIBE access to a requested topic
+ * - {@link Errors#REPLICA_NOT_AVAILABLE} If the request is received by a broker which is not a replica
+ * - {@link Errors#NOT_LEADER_FOR_PARTITION} If the broker is not a leader and either the provided leader epoch
+ *     matches the known leader epoch on the broker or is empty
+ * - {@link Errors#FENCED_LEADER_EPOCH} If the epoch is lower than the broker's epoch
+ * - {@link Errors#UNKNOWN_LEADER_EPOCH} If the epoch is larger than the broker's epoch
+ * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} If the broker does not have metadata for a topic or partition
+ * - {@link Errors#KAFKA_STORAGE_ERROR} If the log directory for one of the requested partitions is offline
+ * - {@link Errors#UNKNOWN_SERVER_ERROR} For any unexpected errors
+ */
 public class OffsetsForLeaderEpochResponse extends AbstractResponse {
     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");
diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala
index 986d352..b2b36af 100644
--- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala
+++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala
@@ -76,4 +76,8 @@ case class BrokerEndPoint(id: Int, host: String, port: Int) {
     4 + /* broker Id */
     4 + /* port */
     shortStringLength(host)
+
+  override def toString: String = {
+    s"BrokerEndPoint(id=$id, host=$host:$port)"
+  }
 }
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index 307fb81..024fdcc 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -16,45 +16,61 @@
  */
 package kafka.cluster
 
+import java.util.Optional
 import java.util.concurrent.locks.ReentrantReadWriteLock
 
 import com.yammer.metrics.core.Gauge
 import kafka.api.{LeaderAndIsr, Request}
 import kafka.common.UnexpectedAppendOffsetException
 import kafka.controller.KafkaController
-import kafka.log.{LogAppendInfo, LogConfig}
+import kafka.log._
 import kafka.metrics.KafkaMetricsGroup
 import kafka.server._
 import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
 import kafka.utils._
-import kafka.zk.AdminZkClient
+import kafka.zk.{AdminZkClient, KafkaZkClient}
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{ReplicaNotAvailableException, NotEnoughReplicasException, NotLeaderForPartitionException, PolicyViolationException}
+import org.apache.kafka.common.errors._
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.protocol.Errors._
-import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
 import org.apache.kafka.common.requests.EpochEndOffset._
-import org.apache.kafka.common.requests.{EpochEndOffset, LeaderAndIsrRequest}
+import org.apache.kafka.common.requests._
 import org.apache.kafka.common.utils.Time
 
 import scala.collection.JavaConverters._
 import scala.collection.Map
 
+object Partition {
+  def apply(topicPartition: TopicPartition,
+            time: Time,
+            replicaManager: ReplicaManager): Partition = {
+    new Partition(topicPartition,
+      isOffline = false,
+      replicaLagTimeMaxMs = replicaManager.config.replicaLagTimeMaxMs,
+      localBrokerId = replicaManager.config.brokerId,
+      time = time,
+      replicaManager = replicaManager,
+      logManager = replicaManager.logManager,
+      zkClient = replicaManager.zkClient)
+  }
+}
+
 /**
  * Data structure that represents a topic partition. The leader maintains the AR, ISR, CUR, RAR
  */
-class Partition(val topic: String,
-                val partitionId: Int,
-                time: Time,
-                replicaManager: ReplicaManager,
-                val isOffline: Boolean = false) extends Logging with KafkaMetricsGroup {
-
-  val topicPartition = new TopicPartition(topic, partitionId)
-
-  // Do not use replicaManager if this partition is ReplicaManager.OfflinePartition
-  private val localBrokerId = if (!isOffline) replicaManager.config.brokerId else -1
-  private val logManager = if (!isOffline) replicaManager.logManager else null
-  private val zkClient = if (!isOffline) replicaManager.zkClient else null
+class Partition(val topicPartition: TopicPartition,
+                val isOffline: Boolean,
+                private val replicaLagTimeMaxMs: Long,
+                private val localBrokerId: Int,
+                private val time: Time,
+                private val replicaManager: ReplicaManager,
+                private val logManager: LogManager,
+                private val zkClient: KafkaZkClient) extends Logging with KafkaMetricsGroup {
+
+  def topic: String = topicPartition.topic
+  def partitionId: Int = topicPartition.partition
+
   // allReplicasMap includes both assigned replicas and the future replica if there is ongoing replica movement
   private val allReplicasMap = new Pool[Int, Replica]
   // The read lock is only required when multiple reads are executed and needs to be in a consistent manner
@@ -155,22 +171,26 @@ class Partition(val topic: String,
     // current replica and the existence of the future replica, no other thread can update the log directory of the
     // current replica or remove the future replica.
     inWriteLock(leaderIsrUpdateLock) {
-      val currentReplica = getReplica().get
+      val currentReplica = localReplicaOrException
       if (currentReplica.log.get.dir.getParent == logDir)
         false
-      else if (getReplica(Request.FutureLocalReplicaId).isDefined) {
-        val futureReplicaLogDir = getReplica(Request.FutureLocalReplicaId).get.log.get.dir.getParent
-        if (futureReplicaLogDir != logDir)
-          throw new IllegalStateException(s"The future log dir $futureReplicaLogDir of $topicPartition is different from the requested log dir $logDir")
-        false
-      } else {
-        getOrCreateReplica(Request.FutureLocalReplicaId)
-        true
+      else {
+        futureLocalReplica match {
+          case Some(replica) =>
+            val futureReplicaLogDir = replica.log.get.dir.getParent
+            if (futureReplicaLogDir != logDir)
+              throw new IllegalStateException(s"The future log dir $futureReplicaLogDir of $topicPartition is " +
+                s"different from the requested log dir $logDir")
+            false
+          case None =>
+            getOrCreateReplica(Request.FutureLocalReplicaId, isNew = false)
+            true
+        }
       }
     }
   }
 
-  def getOrCreateReplica(replicaId: Int = localBrokerId, isNew: Boolean = false): Replica = {
+  def getOrCreateReplica(replicaId: Int, isNew: Boolean = false): Replica = {
     allReplicasMap.getAndMaybePut(replicaId, {
       if (isReplicaLocal(replicaId)) {
         val adminZkClient = new AdminZkClient(zkClient)
@@ -187,14 +207,77 @@ class Partition(val topic: String,
     })
   }
 
-  def getReplica(replicaId: Int = localBrokerId): Option[Replica] = Option(allReplicasMap.get(replicaId))
+  def getReplica(replicaId: Int): Option[Replica] = Option(allReplicasMap.get(replicaId))
+
+  private def checkCurrentLeaderEpoch(remoteLeaderEpochOpt: Optional[Integer]): Errors = {
+    if (!remoteLeaderEpochOpt.isPresent) {
+      Errors.NONE
+    } else {
+      val remoteLeaderEpoch = remoteLeaderEpochOpt.get
+      val localLeaderEpoch = leaderEpoch
+      if (localLeaderEpoch > remoteLeaderEpoch)
+        Errors.FENCED_LEADER_EPOCH
+      else if (localLeaderEpoch < remoteLeaderEpoch)
+        Errors.UNKNOWN_LEADER_EPOCH
+      else
+        Errors.NONE
+    }
+  }
+
+  private def getLocalReplica(replicaId: Int,
+                              currentLeaderEpoch: Optional[Integer],
+                              requireLeader: Boolean): Either[Replica, Errors] = {
+    checkCurrentLeaderEpoch(currentLeaderEpoch) match {
+      case Errors.NONE =>
+        if (requireLeader && !leaderReplicaIdOpt.contains(localBrokerId)) {
+          Right(Errors.NOT_LEADER_FOR_PARTITION)
+        } else {
+          val replica = allReplicasMap.get(replicaId)
+          if (replica == null) {
+            if (requireLeader)
+              Right(Errors.NOT_LEADER_FOR_PARTITION)
+            else
+              Right(Errors.REPLICA_NOT_AVAILABLE)
+          } else {
+            Left(replica)
+          }
+        }
+      case error =>
+        Right(error)
+    }
+  }
+
+  def localReplica: Option[Replica] = getReplica(localBrokerId)
+
+  def localReplicaOrException: Replica = localReplica.getOrElse {
+    throw new ReplicaNotAvailableException(s"Replica for partition $topicPartition is not available " +
+      s"on broker $localBrokerId")
+  }
 
-  def getReplicaOrException(replicaId: Int = localBrokerId): Replica =
-    getReplica(replicaId).getOrElse(
-      throw new ReplicaNotAvailableException(s"Replica $replicaId is not available for partition $topicPartition"))
+  def futureLocalReplica: Option[Replica] = getReplica(Request.FutureLocalReplicaId)
 
-  def leaderReplicaIfLocal: Option[Replica] =
-    leaderReplicaIdOpt.filter(_ == localBrokerId).flatMap(getReplica)
+  def futureLocalReplicaOrException: Replica = futureLocalReplica.getOrElse {
+    throw new ReplicaNotAvailableException(s"Future replica for partition $topicPartition is not available " +
+      s"on broker $localBrokerId")
+  }
+
+  def leaderReplicaIfLocal: Option[Replica] = {
+    if (leaderReplicaIdOpt.contains(localBrokerId))
+      localReplica
+    else
+      None
+  }
+
+  private def localReplicaWithEpochOrException(currentLeaderEpoch: Optional[Integer],
+                                               requireLeader: Boolean): Replica = {
+    getLocalReplica(localBrokerId, currentLeaderEpoch, requireLeader) match {
+      case Left(replica) => replica
+      case Right(error) =>
+        throw error.exception(s"Failed to find ${if (requireLeader) "leader " else ""} for " +
+          s"partition $topicPartition with leader epoch $currentLeaderEpoch. The current leader " +
+          s"is $leaderReplicaIdOpt and the current epoch $leaderEpoch")
+    }
+  }
 
   def addReplicaIfNotExists(replica: Replica): Replica =
     allReplicasMap.putIfNotExists(replica.brokerId, replica)
@@ -211,9 +294,9 @@ class Partition(val topic: String,
 
   def futureReplicaDirChanged(newDestinationDir: String): Boolean = {
     inReadLock(leaderIsrUpdateLock) {
-      getReplica(Request.FutureLocalReplicaId) match {
-        case Some(futureReplica) =>
-          if (futureReplica.log.get.dir.getParent != newDestinationDir)
+      futureLocalReplica match {
+        case Some(replica) =>
+          if (replica.log.get.dir.getParent != newDestinationDir)
             true
           else
             false
@@ -234,13 +317,13 @@ class Partition(val topic: String,
   // Only ReplicaAlterDirThread will call this method and ReplicaAlterDirThread should remove the partition
   // from its partitionStates if this method returns true
   def maybeReplaceCurrentWithFutureReplica(): Boolean = {
-    val replica = getReplica().get
-    val futureReplicaLEO = getReplica(Request.FutureLocalReplicaId).map(_.logEndOffset)
+    val replica = localReplicaOrException
+    val futureReplicaLEO = futureLocalReplica.map(_.logEndOffset)
     if (futureReplicaLEO.contains(replica.logEndOffset)) {
       // The write lock is needed to make sure that while ReplicaAlterDirThread checks the LEO of the
       // current replica, no other thread can update LEO of the current replica via log truncation or log append operation.
       inWriteLock(leaderIsrUpdateLock) {
-        getReplica(Request.FutureLocalReplicaId) match {
+        futureLocalReplica match {
           case Some(futureReplica) =>
             if (replica.logEndOffset == futureReplica.logEndOffset) {
               logManager.replaceCurrentWithFutureLog(topicPartition)
@@ -292,7 +375,7 @@ class Partition(val topic: String,
       inSyncReplicas = newInSyncReplicas
       newAssignedReplicas.foreach(id => getOrCreateReplica(id, partitionStateInfo.isNew))
 
-      val leaderReplica = getReplica().get
+      val leaderReplica = localReplicaOrException
       val leaderEpochStartOffset = leaderReplica.logEndOffset.messageOffset
       info(s"$topicPartition starts at Leader Epoch ${partitionStateInfo.basePartitionState.leaderEpoch} from " +
         s"offset $leaderEpochStartOffset. Previous Leader Epoch was: $leaderEpoch")
@@ -360,12 +443,9 @@ class Partition(val topic: String,
       leaderEpochStartOffsetOpt = None
       zkVersion = partitionStateInfo.basePartitionState.zkVersion
 
-      // If the leader is unchanged and the epochs are no more than one change apart, indicate that no follower changes are required
-      // Otherwise, we missed a leader epoch update, which means the leader's log may have been truncated prior to the current epoch.
-      if (leaderReplicaIdOpt.contains(newLeaderBrokerId) && (leaderEpoch == oldLeaderEpoch || leaderEpoch == oldLeaderEpoch + 1)) {
+      if (leaderReplicaIdOpt.contains(newLeaderBrokerId) && leaderEpoch == oldLeaderEpoch) {
         false
-      }
-      else {
+      } else {
         leaderReplicaIdOpt = Some(newLeaderBrokerId)
         true
       }
@@ -503,7 +583,7 @@ class Partition(val topic: String,
    */
   private def maybeIncrementLeaderHW(leaderReplica: Replica, curTime: Long = time.milliseconds): Boolean = {
     val allLogEndOffsets = assignedReplicas.filter { replica =>
-      curTime - replica.lastCaughtUpTimeMs <= replicaManager.config.replicaLagTimeMaxMs || inSyncReplicas.contains(replica)
+      curTime - replica.lastCaughtUpTimeMs <= replicaLagTimeMaxMs || inSyncReplicas.contains(replica)
     }.map(_.logEndOffset)
     val newHighWatermark = allLogEndOffsets.min(new LogOffsetMetadata.OffsetOrdering)
     val oldHighWatermark = leaderReplica.highWatermark
@@ -606,13 +686,13 @@ class Partition(val topic: String,
       if (isFuture) {
         // Note the replica may be undefined if it is removed by a non-ReplicaAlterLogDirsThread before
         // this method is called
-        getReplica(Request.FutureLocalReplicaId).map { replica =>
+        futureLocalReplica.map { replica =>
           replica.log.get.appendAsFollower(records)
         }
       } else {
         // The read lock is needed to prevent the follower replica from being updated while ReplicaAlterDirThread
         // is executing maybeDeleteAndSwapFutureReplica() to replace follower replica with the future replica.
-        Some(getReplicaOrException().log.get.appendAsFollower(records))
+        Some(localReplicaOrException.log.get.appendAsFollower(records))
       }
     }
   }
@@ -622,7 +702,7 @@ class Partition(val topic: String,
       doAppendRecordsToFollowerOrFutureReplica(records, isFuture)
     } catch {
       case e: UnexpectedAppendOffsetException =>
-        val replica = if (isFuture) getReplicaOrException(Request.FutureLocalReplicaId) else getReplicaOrException()
+        val replica = if (isFuture) futureLocalReplicaOrException else localReplicaOrException
         val logEndOffset = replica.logEndOffset.messageOffset
         if (logEndOffset == replica.logStartOffset &&
             e.firstOffset < logEndOffset && e.lastOffset >= logEndOffset) {
@@ -653,8 +733,8 @@ class Partition(val topic: String,
 
           // Avoid writing to leader if there are not enough insync replicas to make it safe
           if (inSyncSize < minIsr && requiredAcks == -1) {
-            throw new NotEnoughReplicasException("Number of insync replicas for partition %s is [%d], below required minimum [%d]"
-              .format(topicPartition, inSyncSize, minIsr))
+            throw new NotEnoughReplicasException(s"The size of the current ISR ${inSyncReplicas.map(_.brokerId)} " +
+              s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition")
           }
 
           val info = log.appendAsLeader(records, leaderEpoch = this.leaderEpoch, isFromClient)
@@ -676,6 +756,118 @@ class Partition(val topic: String,
     info
   }
 
+  def readRecords(fetchOffset: Long,
+                  currentLeaderEpoch: Optional[Integer],
+                  maxBytes: Int,
+                  fetchIsolation: FetchIsolation,
+                  fetchOnlyFromLeader: Boolean,
+                  minOneMessage: Boolean): LogReadInfo = inReadLock(leaderIsrUpdateLock) {
+    // decide whether to only fetch from leader
+    val localReplica = localReplicaWithEpochOrException(currentLeaderEpoch, fetchOnlyFromLeader)
+
+    /* Read the LogOffsetMetadata prior to performing the read from the log.
+     * We use the LogOffsetMetadata to determine if a particular replica is in-sync or not.
+     * Using the log end offset after performing the read can lead to a race condition
+     * where data gets appended to the log immediately after the replica has consumed from it
+     * This can cause a replica to always be out of sync.
+     */
+    val initialHighWatermark = localReplica.highWatermark.messageOffset
+    val initialLogStartOffset = localReplica.logStartOffset
+    val initialLogEndOffset = localReplica.logEndOffset.messageOffset
+    val initialLastStableOffset = localReplica.lastStableOffset.messageOffset
+
+    val maxOffsetOpt = fetchIsolation match {
+      case FetchLogEnd => None
+      case FetchHighWatermark => Some(initialHighWatermark)
+      case FetchTxnCommitted => Some(initialLastStableOffset)
+    }
+
+    val fetchedData = localReplica.log match {
+      case Some(log) =>
+        log.read(fetchOffset, maxBytes, maxOffsetOpt, minOneMessage,
+          includeAbortedTxns = fetchIsolation == FetchTxnCommitted)
+
+      case None =>
+        error(s"Leader does not have a local log")
+        FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)
+    }
+
+    LogReadInfo(
+      fetchedData = fetchedData,
+      highWatermark = initialHighWatermark,
+      logStartOffset = initialLogStartOffset,
+      logEndOffset = initialLogEndOffset,
+      lastStableOffset = initialLastStableOffset)
+  }
+
+  def fetchOffsetForTimestamp(timestamp: Long,
+                              isolationLevel: Option[IsolationLevel],
+                              currentLeaderEpoch: Optional[Integer],
+                              fetchOnlyFromLeader: Boolean): TimestampOffset = inReadLock(leaderIsrUpdateLock) {
+    // decide whether to only fetch from leader
+    val localReplica = localReplicaWithEpochOrException(currentLeaderEpoch, fetchOnlyFromLeader)
+
+    val lastFetchableOffset = isolationLevel match {
+      case Some(IsolationLevel.READ_COMMITTED) => localReplica.lastStableOffset.messageOffset
+      case Some(IsolationLevel.READ_UNCOMMITTED) => localReplica.highWatermark.messageOffset
+      case None => localReplica.logEndOffset.messageOffset
+    }
+
+    if (timestamp == ListOffsetRequest.LATEST_TIMESTAMP) {
+      TimestampOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset)
+    } else {
+      def allowed(timestampOffset: TimestampOffset): Boolean =
+        timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP || timestampOffset.offset < lastFetchableOffset
+
+      val fetchedOffset = logManager.getLog(topicPartition).flatMap { log =>
+        log.fetchOffsetsByTimestamp(timestamp)
+      }
+
+      fetchedOffset.filter(allowed).getOrElse(TimestampOffset.Unknown)
+    }
+  }
+
+  def fetchOffsetSnapshot(currentLeaderEpoch: Optional[Integer],
+                          fetchOnlyFromLeader: Boolean): LogOffsetSnapshot = inReadLock(leaderIsrUpdateLock) {
+    // decide whether to only fetch from leader
+    val localReplica = localReplicaWithEpochOrException(currentLeaderEpoch, fetchOnlyFromLeader)
+    localReplica.offsetSnapshot
+  }
+
+  def fetchOffsetSnapshotOrError(currentLeaderEpoch: Optional[Integer],
+                                 fetchOnlyFromLeader: Boolean): Either[LogOffsetSnapshot, Errors] = {
+    inReadLock(leaderIsrUpdateLock) {
+      getLocalReplica(localBrokerId, currentLeaderEpoch, fetchOnlyFromLeader)
+        .left.map(_.offsetSnapshot)
+    }
+  }
+
+  def legacyFetchOffsetsForTimestamp(timestamp: Long,
+                                     maxNumOffsets: Int,
+                                     isFromConsumer: Boolean,
+                                     fetchOnlyFromLeader: Boolean): Seq[Long] = inReadLock(leaderIsrUpdateLock) {
+    val localReplica = localReplicaWithEpochOrException(Optional.empty(), fetchOnlyFromLeader)
+    val allOffsets = logManager.getLog(topicPartition) match {
+      case Some(log) =>
+        log.legacyFetchOffsetsBefore(timestamp, maxNumOffsets)
+      case None =>
+        if (timestamp == ListOffsetRequest.LATEST_TIMESTAMP || timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
+          Seq(0L)
+        else
+          Nil
+    }
+
+    if (!isFromConsumer) {
+      allOffsets
+    } else {
+      val hw = localReplica.highWatermark.messageOffset
+      if (allOffsets.exists(_ > hw))
+        hw +: allOffsets.dropWhile(_ > hw)
+      else
+        allOffsets
+    }
+  }
+
   def logStartOffset: Long = {
     inReadLock(leaderIsrUpdateLock) {
       leaderReplicaIfLocal.map(_.log.get.logStartOffset).getOrElse(-1)
@@ -688,18 +880,26 @@ class Partition(val topic: String,
    *
    * Return low watermark of the partition.
    */
-  def deleteRecordsOnLeader(offset: Long): Long = {
-    inReadLock(leaderIsrUpdateLock) {
-      leaderReplicaIfLocal match {
-        case Some(leaderReplica) =>
-          if (!leaderReplica.log.get.config.delete)
-            throw new PolicyViolationException("Records of partition %s can not be deleted due to the configured policy".format(topicPartition))
-          leaderReplica.maybeIncrementLogStartOffset(offset)
-          lowWatermarkIfLeader
-        case None =>
-          throw new NotLeaderForPartitionException("Leader not local for partition %s on broker %d"
-            .format(topicPartition, localBrokerId))
-      }
+  def deleteRecordsOnLeader(offset: Long): LogDeleteRecordsResult = inReadLock(leaderIsrUpdateLock) {
+    leaderReplicaIfLocal match {
+      case Some(leaderReplica) =>
+        if (!leaderReplica.log.get.config.delete)
+          throw new PolicyViolationException(s"Records of partition $topicPartition can not be deleted due to the configured policy")
+
+        val convertedOffset = if (offset == DeleteRecordsRequest.HIGH_WATERMARK)
+          leaderReplica.highWatermark.messageOffset
+        else
+          offset
+
+        if (convertedOffset < 0)
+          throw new OffsetOutOfRangeException(s"The offset $convertedOffset for partition $topicPartition is not valid")
+
+        leaderReplica.maybeIncrementLogStartOffset(convertedOffset)
+        LogDeleteRecordsResult(
+          requestedOffset = convertedOffset,
+          lowWatermark = lowWatermarkIfLeader)
+      case None =>
+        throw new NotLeaderForPartitionException(s"Leader not local for partition $topicPartition on broker $localBrokerId")
     }
   }
 
@@ -732,21 +932,29 @@ class Partition(val topic: String,
   }
 
   /**
-    * @param leaderEpoch Requested leader epoch
-    * @return The requested leader epoch and the end offset of this leader epoch, or if the requested
-    *         leader epoch is unknown, the leader epoch less than the requested leader epoch and the end offset
-    *         of this leader epoch. The end offset of a leader epoch is defined as the start
-    *         offset of the first leader epoch larger than the leader epoch, or else the log end
-    *         offset if the leader epoch is the latest leader epoch.
-    */
-  def lastOffsetForLeaderEpoch(leaderEpoch: Int): EpochEndOffset = {
+   * Find the (exclusive) last offset of the largest epoch less than or equal to the requested epoch.
+   *
+   * @param currentLeaderEpoch The expected epoch of the current leader (if known)
+   * @param leaderEpoch Requested leader epoch
+   * @param fetchOnlyFromLeader Whether or not to require servicing only from the leader
+   *
+   * @return The requested leader epoch and the end offset of this leader epoch, or if the requested
+   *         leader epoch is unknown, the leader epoch less than the requested leader epoch and the end offset
+   *         of this leader epoch. The end offset of a leader epoch is defined as the start
+   *         offset of the first leader epoch larger than the leader epoch, or else the log end
+   *         offset if the leader epoch is the latest leader epoch.
+   */
+  def lastOffsetForLeaderEpoch(currentLeaderEpoch: Optional[Integer],
+                               leaderEpoch: Int,
+                               fetchOnlyFromLeader: Boolean): EpochEndOffset = {
     inReadLock(leaderIsrUpdateLock) {
-      leaderReplicaIfLocal match {
-        case Some(leaderReplica) =>
-          val (epoch, offset) = leaderReplica.epochs.get.endOffsetFor(leaderEpoch)
+      val localReplicaOrError = getLocalReplica(localBrokerId, currentLeaderEpoch, fetchOnlyFromLeader)
+      localReplicaOrError match {
+        case Left(replica) =>
+          val (epoch, offset) = replica.epochs.get.endOffsetFor(leaderEpoch)
           new EpochEndOffset(NONE, epoch, offset)
-        case None =>
-          new EpochEndOffset(NOT_LEADER_FOR_PARTITION, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
+        case Right(error) =>
+          new EpochEndOffset(error, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
       }
     }
   }
diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala
index 22860c7..b8e831a 100644
--- a/core/src/main/scala/kafka/cluster/Replica.scala
+++ b/core/src/main/scala/kafka/cluster/Replica.scala
@@ -17,8 +17,8 @@
 
 package kafka.cluster
 
-import kafka.log.Log
 import kafka.server.epoch.LeaderEpochFileCache
+import kafka.log.{Log, LogOffsetSnapshot}
 import kafka.utils.Logging
 import kafka.server.{LogOffsetMetadata, LogReadResult}
 import org.apache.kafka.common.{KafkaException, TopicPartition}
@@ -185,6 +185,14 @@ class Replica(val brokerId: Int,
     }
   }
 
+  def offsetSnapshot: LogOffsetSnapshot = {
+    LogOffsetSnapshot(
+      logStartOffset = logStartOffset,
+      logEndOffset = logEndOffset,
+      highWatermark =  highWatermark,
+      lastStableOffset = lastStableOffset)
+  }
+
   override def equals(that: Any): Boolean = that match {
     case other: Replica => brokerId == other.brokerId && topicPartition == other.topicPartition
     case _ => false
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
index 31cd361..626aaad 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
@@ -29,7 +29,7 @@ import com.yammer.metrics.core.Gauge
 import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0, KAFKA_2_1_IV0, KAFKA_2_1_IV1}
 import kafka.common.{MessageFormatter, OffsetAndMetadata}
 import kafka.metrics.KafkaMetricsGroup
-import kafka.server.ReplicaManager
+import kafka.server.{FetchHighWatermark, FetchLogEnd, ReplicaManager}
 import kafka.utils.CoreUtils.inLock
 import kafka.utils._
 import kafka.zk.KafkaZkClient
@@ -510,7 +510,7 @@ class GroupMetadataManager(brokerId: Int,
   }
 
   private def doLoadGroupsAndOffsets(topicPartition: TopicPartition, onGroupLoaded: GroupMetadata => Unit) {
-    def highWaterMark = replicaManager.getLogEndOffset(topicPartition).getOrElse(-1L)
+    def logEndOffset: Long = replicaManager.getLogEndOffset(topicPartition).getOrElse(-1L)
 
     replicaManager.getLog(topicPartition) match {
       case None =>
@@ -528,9 +528,9 @@ class GroupMetadataManager(brokerId: Int,
         val loadedGroups = mutable.Map[String, GroupMetadata]()
         val removedGroups = mutable.Set[String]()
 
-        while (currOffset < highWaterMark && !shuttingDown.get()) {
+        while (currOffset < logEndOffset && !shuttingDown.get()) {
           val fetchDataInfo = log.read(currOffset, config.loadBufferSize, maxOffset = None,
-            minOneMessage = true, isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+            minOneMessage = true, includeAbortedTxns = false)
           val memRecords = fetchDataInfo.records match {
             case records: MemoryRecords => records
             case fileRecords: FileRecords =>
diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
index 50d96c3..574c64e 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
@@ -308,7 +308,7 @@ class TransactionStateManager(brokerId: Int,
             && inReadLock(stateLock) {loadingPartitions.exists { idAndEpoch: TransactionPartitionAndLeaderEpoch =>
               idAndEpoch.txnPartitionId == topicPartition.partition && idAndEpoch.coordinatorEpoch == coordinatorEpoch}}) {
             val fetchDataInfo = log.read(currOffset, config.transactionLogLoadBufferSize, maxOffset = None,
-              minOneMessage = true, isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+              minOneMessage = true, includeAbortedTxns = false)
             val memRecords = fetchDataInfo.records match {
               case records: MemoryRecords => records
               case fileRecords: FileRecords =>
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index 699d3d1..8915c14 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -38,7 +38,7 @@ import kafka.utils._
 import org.apache.kafka.common.errors._
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
-import org.apache.kafka.common.requests.{IsolationLevel, ListOffsetRequest}
+import org.apache.kafka.common.requests.ListOffsetRequest
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.common.{KafkaException, TopicPartition}
 
@@ -107,6 +107,25 @@ case class LogAppendInfo(var firstOffset: Option[Long],
 }
 
 /**
+ * Container class which represents a snapshot of the significant offsets for a partition. This allows fetching
+ * of these offsets atomically without the possibility of a leader change affecting their consistency relative
+ * to each other. See [[kafka.cluster.Partition.fetchOffsetSnapshot()]].
+ */
+case class LogOffsetSnapshot(logStartOffset: Long,
+                             logEndOffset: LogOffsetMetadata,
+                             highWatermark: LogOffsetMetadata,
+                             lastStableOffset: LogOffsetMetadata)
+
+/**
+ * Another container which is used for lower level reads using  [[kafka.cluster.Partition.readRecords()]].
+ */
+case class LogReadInfo(fetchedData: FetchDataInfo,
+                       highWatermark: Long,
+                       logStartOffset: Long,
+                       logEndOffset: Long,
+                       lastStableOffset: Long)
+
+/**
  * A class used to hold useful metadata about a completed transaction. This is used to build
  * the transaction index after appending to the log.
  *
@@ -1087,11 +1106,6 @@ class Log(@volatile var dir: File,
     }
   }
 
-  private[log] def readUncommitted(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None,
-                                   minOneMessage: Boolean = false): FetchDataInfo = {
-    read(startOffset, maxLength, maxOffset, minOneMessage, isolationLevel = IsolationLevel.READ_UNCOMMITTED)
-  }
-
   /**
    * Read messages from the log.
    *
@@ -1099,18 +1113,15 @@ class Log(@volatile var dir: File,
    * @param maxLength The maximum number of bytes to read
    * @param maxOffset The offset to read up to, exclusive. (i.e. this offset NOT included in the resulting message set)
    * @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxLength` (if one exists)
-   * @param isolationLevel The isolation level of the fetcher. The READ_UNCOMMITTED isolation level has the traditional
-   *                       read semantics (e.g. consumers are limited to fetching up to the high watermark). In
-   *                       READ_COMMITTED, consumers are limited to fetching up to the last stable offset. Additionally,
-   *                       in READ_COMMITTED, the transaction index is consulted after fetching to collect the list
-   *                       of aborted transactions in the fetch range which the consumer uses to filter the fetched
-   *                       records before they are returned to the user. Note that fetches from followers always use
-   *                       READ_UNCOMMITTED.
+   * @param includeAbortedTxns Whether or not to lookup aborted transactions for fetched data
    * @throws OffsetOutOfRangeException If startOffset is beyond the log end offset or before the log start offset
    * @return The fetch data information including fetch starting offset metadata and messages read.
    */
-  def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None, minOneMessage: Boolean = false,
-           isolationLevel: IsolationLevel): FetchDataInfo = {
+  def read(startOffset: Long,
+           maxLength: Int,
+           maxOffset: Option[Long],
+           minOneMessage: Boolean,
+           includeAbortedTxns: Boolean): FetchDataInfo = {
     maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") {
       trace(s"Reading $maxLength bytes from offset $startOffset of length $size bytes")
 
@@ -1120,7 +1131,7 @@ class Log(@volatile var dir: File,
       val next = currentNextOffsetMetadata.messageOffset
       if (startOffset == next) {
         val abortedTransactions =
-          if (isolationLevel == IsolationLevel.READ_COMMITTED) Some(List.empty[AbortedTransaction])
+          if (includeAbortedTxns) Some(List.empty[AbortedTransaction])
           else None
         return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false,
           abortedTransactions = abortedTransactions)
@@ -1160,10 +1171,10 @@ class Log(@volatile var dir: File,
         if (fetchInfo == null) {
           segmentEntry = segments.higherEntry(segmentEntry.getKey)
         } else {
-          return isolationLevel match {
-            case IsolationLevel.READ_UNCOMMITTED => fetchInfo
-            case IsolationLevel.READ_COMMITTED => addAbortedTransactions(startOffset, segmentEntry, fetchInfo)
-          }
+          return if (includeAbortedTxns)
+            addAbortedTransactions(startOffset, segmentEntry, fetchInfo)
+          else
+            fetchInfo
         }
       }
 
@@ -1266,13 +1277,62 @@ class Log(@volatile var dir: File,
     }
   }
 
+  def legacyFetchOffsetsBefore(timestamp: Long, maxNumOffsets: Int): Seq[Long] = {
+    // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides
+    // constant time access while being safe to use with concurrent collections unlike `toArray`.
+    val segments = logSegments.toBuffer
+    val lastSegmentHasSize = segments.last.size > 0
+
+    val offsetTimeArray =
+      if (lastSegmentHasSize)
+        new Array[(Long, Long)](segments.length + 1)
+      else
+        new Array[(Long, Long)](segments.length)
+
+    for (i <- segments.indices)
+      offsetTimeArray(i) = (math.max(segments(i).baseOffset, logStartOffset), segments(i).lastModified)
+    if (lastSegmentHasSize)
+      offsetTimeArray(segments.length) = (logEndOffset, time.milliseconds)
+
+    var startIndex = -1
+    timestamp match {
+      case ListOffsetRequest.LATEST_TIMESTAMP =>
+        startIndex = offsetTimeArray.length - 1
+      case ListOffsetRequest.EARLIEST_TIMESTAMP =>
+        startIndex = 0
+      case _ =>
+        var isFound = false
+        debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2)))
+        startIndex = offsetTimeArray.length - 1
+        while (startIndex >= 0 && !isFound) {
+          if (offsetTimeArray(startIndex)._2 <= timestamp)
+            isFound = true
+          else
+            startIndex -= 1
+        }
+    }
+
+    val retSize = maxNumOffsets.min(startIndex + 1)
+    val ret = new Array[Long](retSize)
+    for (j <- 0 until retSize) {
+      ret(j) = offsetTimeArray(startIndex)._1
+      startIndex -= 1
+    }
+    // ensure that the returned seq is in descending order of offsets
+    ret.toSeq.sortBy(-_)
+  }
+
   /**
    * Given a message offset, find its corresponding offset metadata in the log.
    * If the message offset is out of range, return None to the caller.
    */
   def convertToOffsetMetadata(offset: Long): Option[LogOffsetMetadata] = {
     try {
-      val fetchDataInfo = readUncommitted(offset, 1)
+      val fetchDataInfo = read(offset,
+        maxLength = 1,
+        maxOffset = None,
+        minOneMessage = false,
+        includeAbortedTxns = false)
       Some(fetchDataInfo.fetchOffsetMetadata)
     } catch {
       case _: OffsetOutOfRangeException => None
diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
index 80d07e5..8d3ac4d 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
@@ -27,11 +27,11 @@ import org.apache.kafka.common.utils.Utils
 import scala.collection.mutable
 import scala.collection.{Map, Set}
 
-abstract class AbstractFetcherManager(protected val name: String, clientId: String, numFetchers: Int = 1)
+abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: String, clientId: String, numFetchers: Int)
   extends Logging with KafkaMetricsGroup {
   // map of (source broker_id, fetcher_id per source broker) => fetcher.
   // package private for test
-  private[server] val fetcherThreadMap = new mutable.HashMap[BrokerIdAndFetcherId, AbstractFetcherThread]
+  private[server] val fetcherThreadMap = new mutable.HashMap[BrokerIdAndFetcherId, T]
   private val lock = new Object
   private var numFetchersPerBroker = numFetchers
   this.logIdent = "[" + name + "] "
@@ -90,17 +90,27 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
     }
   }
 
+  // Visible for testing
+  private[server] def getFetcher(topicPartition: TopicPartition): Option[T] = {
+    lock synchronized {
+      val fetcherId = getFetcherId(topicPartition)
+      fetcherThreadMap.values.find { fetcherThread =>
+        fetcherThread.fetchState(topicPartition).isDefined
+      }
+    }
+  }
+
   // Visibility for testing
-  private[server] def getFetcherId(topic: String, partitionId: Int) : Int = {
+  private[server] def getFetcherId(topicPartition: TopicPartition): Int = {
     lock synchronized {
-      Utils.abs(31 * topic.hashCode() + partitionId) % numFetchersPerBroker
+      Utils.abs(31 * topicPartition.topic.hashCode() + topicPartition.partition) % numFetchersPerBroker
     }
   }
 
   // This method is only needed by ReplicaAlterDirManager
   def markPartitionsForTruncation(brokerId: Int, topicPartition: TopicPartition, truncationOffset: Long) {
     lock synchronized {
-      val fetcherId = getFetcherId(topicPartition.topic, topicPartition.partition)
+      val fetcherId = getFetcherId(topicPartition)
       val brokerIdAndFetcherId = BrokerIdAndFetcherId(brokerId, fetcherId)
       fetcherThreadMap.get(brokerIdAndFetcherId).foreach { thread =>
         thread.markPartitionsForTruncation(topicPartition, truncationOffset)
@@ -109,24 +119,27 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
   }
 
   // to be defined in subclass to create a specific fetcher
-  def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread
+  def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): T
 
-  def addFetcherForPartitions(partitionAndOffsets: Map[TopicPartition, BrokerAndInitialOffset]) {
+  def addFetcherForPartitions(partitionAndOffsets: Map[TopicPartition, InitialFetchState]) {
     lock synchronized {
-      val partitionsPerFetcher = partitionAndOffsets.groupBy { case(topicPartition, brokerAndInitialFetchOffset) =>
-        BrokerAndFetcherId(brokerAndInitialFetchOffset.broker, getFetcherId(topicPartition.topic, topicPartition.partition))}
+      val partitionsPerFetcher = partitionAndOffsets.groupBy { case (topicPartition, brokerAndInitialFetchOffset) =>
+        BrokerAndFetcherId(brokerAndInitialFetchOffset.leader, getFetcherId(topicPartition))
+      }
 
-      def addAndStartFetcherThread(brokerAndFetcherId: BrokerAndFetcherId, brokerIdAndFetcherId: BrokerIdAndFetcherId) {
+      def addAndStartFetcherThread(brokerAndFetcherId: BrokerAndFetcherId, brokerIdAndFetcherId: BrokerIdAndFetcherId): AbstractFetcherThread = {
         val fetcherThread = createFetcherThread(brokerAndFetcherId.fetcherId, brokerAndFetcherId.broker)
         fetcherThreadMap.put(brokerIdAndFetcherId, fetcherThread)
-        fetcherThread.start
+        fetcherThread.start()
+        fetcherThread
       }
 
       for ((brokerAndFetcherId, initialFetchOffsets) <- partitionsPerFetcher) {
         val brokerIdAndFetcherId = BrokerIdAndFetcherId(brokerAndFetcherId.broker.id, brokerAndFetcherId.fetcherId)
-        fetcherThreadMap.get(brokerIdAndFetcherId) match {
-          case Some(f) if f.sourceBroker.host == brokerAndFetcherId.broker.host && f.sourceBroker.port == brokerAndFetcherId.broker.port =>
+        val fetcherThread = fetcherThreadMap.get(brokerIdAndFetcherId) match {
+          case Some(currentFetcherThread) if currentFetcherThread.sourceBroker == brokerAndFetcherId.broker =>
             // reuse the fetcher thread
+            currentFetcherThread
           case Some(f) =>
             f.shutdown()
             addAndStartFetcherThread(brokerAndFetcherId, brokerIdAndFetcherId)
@@ -134,14 +147,14 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
             addAndStartFetcherThread(brokerAndFetcherId, brokerIdAndFetcherId)
         }
 
-        fetcherThreadMap(brokerIdAndFetcherId).addPartitions(initialFetchOffsets.map { case (tp, brokerAndInitOffset) =>
-          tp -> brokerAndInitOffset.initOffset
-        })
+        val initialOffsetAndEpochs = initialFetchOffsets.map { case (tp, brokerAndInitOffset) =>
+          tp -> OffsetAndEpoch(brokerAndInitOffset.initOffset, brokerAndInitOffset.currentLeaderEpoch)
+        }
+
+        fetcherThread.addPartitions(initialOffsetAndEpochs)
+        info(s"Added fetcher to broker ${brokerAndFetcherId.broker} for partitions $initialOffsetAndEpochs")
       }
     }
-
-    info("Added fetcher for partitions %s".format(partitionAndOffsets.map { case (topicPartition, brokerAndInitialOffset) =>
-      "[" + topicPartition + ", initOffset " + brokerAndInitialOffset.initOffset + " to broker " + brokerAndInitialOffset.broker + "] "}))
   }
 
   def removeFetcherForPartitions(partitions: Set[TopicPartition]) {
@@ -149,7 +162,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
       for (fetcher <- fetcherThreadMap.values)
         fetcher.removePartitions(partitions)
     }
-    info("Removed fetcher for partitions %s".format(partitions.mkString(",")))
+    info(s"Removed fetcher for partitions $partitions")
   }
 
   def shutdownIdleFetcherThreads() {
@@ -181,6 +194,6 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
 
 case class BrokerAndFetcherId(broker: BrokerEndPoint, fetcherId: Int)
 
-case class BrokerAndInitialOffset(broker: BrokerEndPoint, initOffset: Long)
+case class InitialFetchState(leader: BrokerEndPoint, currentLeaderEpoch: Int, initOffset: Long)
 
 case class BrokerIdAndFetcherId(brokerId: Int, fetcherId: Int)
diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index 4a2719e..60d397d 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -18,11 +18,12 @@
 package kafka.server
 
 import java.nio.ByteBuffer
+import java.util.Optional
 import java.util.concurrent.locks.ReentrantLock
 
 import kafka.cluster.BrokerEndPoint
 import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
-import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException}
+import org.apache.kafka.common.errors._
 import org.apache.kafka.common.requests.EpochEndOffset._
 import kafka.common.ClientIdAndBroker
 import kafka.metrics.KafkaMetricsGroup
@@ -40,7 +41,8 @@ import kafka.log.LogAppendInfo
 import org.apache.kafka.common.{KafkaException, TopicPartition}
 import org.apache.kafka.common.internals.PartitionStates
 import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Records}
-import org.apache.kafka.common.requests.{EpochEndOffset, FetchRequest, FetchResponse}
+import org.apache.kafka.common.requests._
+
 
 import scala.math._
 
@@ -54,9 +56,10 @@ abstract class AbstractFetcherThread(name: String,
                                      isInterruptible: Boolean = true)
   extends ShutdownableThread(name, isInterruptible) {
 
-  type PD = FetchResponse.PartitionData[Records]
+  type FetchData = FetchResponse.PartitionData[Records]
+  type EpochData = OffsetsForLeaderEpochRequest.PartitionData
 
-  private[server] val partitionStates = new PartitionStates[PartitionFetchState]
+  private val partitionStates = new PartitionStates[PartitionFetchState]
   private val partitionMapLock = new ReentrantLock
   private val partitionMapCond = partitionMapLock.newCondition()
 
@@ -69,7 +72,7 @@ abstract class AbstractFetcherThread(name: String,
   // process fetched data
   protected def processPartitionData(topicPartition: TopicPartition,
                                      fetchOffset: Long,
-                                     partitionData: PD): Option[LogAppendInfo]
+                                     partitionData: FetchData): Option[LogAppendInfo]
 
   protected def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit
 
@@ -83,13 +86,13 @@ abstract class AbstractFetcherThread(name: String,
 
   protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch]
 
-  protected def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset]
+  protected def fetchEpochsFromLeader(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset]
 
-  protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, PD)]
+  protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, FetchData)]
 
-  protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition): Long
+  protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long
 
-  protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition): Long
+  protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long
 
   override def shutdown() {
     initiateShutdown()
@@ -138,16 +141,19 @@ abstract class AbstractFetcherThread(name: String,
    * Builds offset for leader epoch requests for partitions that are in the truncating phase based
    * on latest epochs of the future replicas (the one that is fetching)
    */
-  private def buildLeaderEpochRequest(): ResultWithPartitions[Map[TopicPartition, Int]] = inLock(partitionMapLock) {
+  private def buildLeaderEpochRequest(): ResultWithPartitions[Map[TopicPartition, EpochData]] = inLock(partitionMapLock) {
     var partitionsWithoutEpochs = mutable.Set.empty[TopicPartition]
-    var partitionsWithEpochs = mutable.Map.empty[TopicPartition, Int]
+    var partitionsWithEpochs = mutable.Map.empty[TopicPartition, EpochData]
 
     partitionStates.partitionStates.asScala.foreach { state =>
       val tp = state.topicPartition
-      if (state.value.isTruncatingLog) {
+      if (state.value.isTruncating) {
         latestEpoch(tp) match {
-          case Some(latestEpoch) => partitionsWithEpochs += tp -> latestEpoch
-          case None => partitionsWithoutEpochs += tp
+          case Some(latestEpoch) =>
+            val partitionData = new EpochData(Optional.of(state.value.currentLeaderEpoch), latestEpoch)
+            partitionsWithEpochs += tp -> partitionData
+          case None =>
+            partitionsWithoutEpochs += tp
         }
       }
     }
@@ -184,16 +190,21 @@ abstract class AbstractFetcherThread(name: String,
 
   private def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): ResultWithPartitions[Map[TopicPartition, OffsetTruncationState]] = {
     val fetchOffsets = mutable.HashMap.empty[TopicPartition, OffsetTruncationState]
-    val partitionsWithError = mutable.Set[TopicPartition]()
+    val partitionsWithError = mutable.HashSet.empty[TopicPartition]
 
     fetchedEpochs.foreach { case (tp, leaderEpochOffset) =>
       try {
-        if (leaderEpochOffset.hasError) {
-          info(s"Retrying leaderEpoch request for partition $tp as the leader reported an error: ${leaderEpochOffset.error}")
-          partitionsWithError += tp
-        } else {
-          val offsetTruncationState = truncate(tp, leaderEpochOffset)
-          fetchOffsets.put(tp, offsetTruncationState)
+        leaderEpochOffset.error match {
+          case Errors.NONE =>
+            val offsetTruncationState = truncate(tp, leaderEpochOffset)
+            fetchOffsets.put(tp, offsetTruncationState)
+
+          case Errors.FENCED_LEADER_EPOCH =>
+            onPartitionFenced(tp)
+
+          case error =>
+            info(s"Retrying leaderEpoch request for partition $tp as the leader reported an error: $error")
+            partitionsWithError += tp
         }
       } catch {
         case e: KafkaStorageException =>
@@ -205,6 +216,15 @@ abstract class AbstractFetcherThread(name: String,
     ResultWithPartitions(fetchOffsets, partitionsWithError)
   }
 
+  private def onPartitionFenced(tp: TopicPartition): Unit = inLock(partitionMapLock) {
+    Option(partitionStates.stateValue(tp)).foreach { currentFetchState =>
+      val currentLeaderEpoch = currentFetchState.currentLeaderEpoch
+      info(s"Partition $tp has an older epoch ($currentLeaderEpoch) than the current leader. Will await " +
+        s"the new LeaderAndIsr state before resuming fetching.")
+      partitionStates.remove(tp)
+    }
+  }
+
   private def truncate(topicPartition: TopicPartition, epochEndOffset: EpochEndOffset): OffsetTruncationState = {
     val offsetTruncationState = getOffsetTruncationState(topicPartition, epochEndOffset)
     truncate(topicPartition, offsetTruncationState)
@@ -214,7 +234,7 @@ abstract class AbstractFetcherThread(name: String,
   private def processFetchRequest(fetchStates: Map[TopicPartition, PartitionFetchState],
                                   fetchRequest: FetchRequest.Builder): Unit = {
     val partitionsWithError = mutable.Set[TopicPartition]()
-    var responseData: Seq[(TopicPartition, PD)] = Seq.empty
+    var responseData: Seq[(TopicPartition, FetchData)] = Seq.empty
 
     try {
       trace(s"Sending fetch request $fetchRequest")
@@ -238,17 +258,17 @@ abstract class AbstractFetcherThread(name: String,
       // process fetched data
       inLock(partitionMapLock) {
         responseData.foreach { case (topicPartition, partitionData) =>
-          Option(partitionStates.stateValue(topicPartition)).foreach { currentPartitionFetchState =>
+          Option(partitionStates.stateValue(topicPartition)).foreach { currentFetchState =>
             // It's possible that a partition is removed and re-added or truncated when there is a pending fetch request.
             // In this case, we only want to process the fetch response if the partition state is ready for fetch and
             // the current offset is the same as the offset requested.
-            val fetchOffset = fetchStates(topicPartition).fetchOffset
-            if (fetchOffset == currentPartitionFetchState.fetchOffset && currentPartitionFetchState.isReadyForFetch) {
+            val fetchState = fetchStates(topicPartition)
+            if (fetchState.fetchOffset == currentFetchState.fetchOffset && currentFetchState.isReadyForFetch) {
               partitionData.error match {
                 case Errors.NONE =>
                   try {
                     // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
-                    val logAppendInfoOpt = processPartitionData(topicPartition, currentPartitionFetchState.fetchOffset,
+                    val logAppendInfoOpt = processPartitionData(topicPartition, currentFetchState.fetchOffset,
                       partitionData)
 
                     logAppendInfoOpt.foreach { logAppendInfo =>
@@ -259,7 +279,9 @@ abstract class AbstractFetcherThread(name: String,
                       // ReplicaDirAlterThread may have removed topicPartition from the partitionStates after processing the partition data
                       if (validBytes > 0 && partitionStates.contains(topicPartition)) {
                         // Update partitionStates only if there is no exception during processPartitionData
-                        partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(nextOffset))
+                        val newFetchState = PartitionFetchState(nextOffset, fetchState.currentLeaderEpoch,
+                          state = Fetching)
+                        partitionStates.updateAndMoveToEnd(topicPartition, newFetchState)
                         fetcherStats.byteRate.mark(validBytes)
                       }
                     }
@@ -271,34 +293,34 @@ abstract class AbstractFetcherThread(name: String,
                       // 2. If the message is corrupt due to a transient state in the log (truncation, partial writes
                       //    can cause this), we simply continue and should get fixed in the subsequent fetches
                       error(s"Found invalid messages during fetch for partition $topicPartition " +
-                        s"offset ${currentPartitionFetchState.fetchOffset}", ime)
+                        s"offset ${currentFetchState.fetchOffset}", ime)
                       partitionsWithError += topicPartition
                     case e: KafkaStorageException =>
                       error(s"Error while processing data for partition $topicPartition", e)
                       partitionsWithError += topicPartition
                     case e: Throwable =>
                       throw new KafkaException(s"Error processing data for partition $topicPartition " +
-                        s"offset ${currentPartitionFetchState.fetchOffset}", e)
+                        s"offset ${currentFetchState.fetchOffset}", e)
                   }
                 case Errors.OFFSET_OUT_OF_RANGE =>
-                  try {
-                    val newOffset = handleOffsetOutOfRange(topicPartition)
-                    partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
-                    info(s"Current offset ${currentPartitionFetchState.fetchOffset} for partition $topicPartition is " +
-                      s"out of range, which typically implies a leader change. Reset fetch offset to $newOffset")
-                  } catch {
-                    case e: Throwable =>
-                      error(s"Error getting offset for partition $topicPartition", e)
-                      partitionsWithError += topicPartition
-                  }
+                  if (!handleOutOfRangeError(topicPartition, currentFetchState))
+                    partitionsWithError += topicPartition
+
+                case Errors.UNKNOWN_LEADER_EPOCH =>
+                  debug(s"Remote broker has a smaller leader epoch for partition $topicPartition than " +
+                    s"this replica's current leader epoch of ${fetchState.currentLeaderEpoch}.")
+                  partitionsWithError += topicPartition
+
+                case Errors.FENCED_LEADER_EPOCH =>
+                  onPartitionFenced(topicPartition)
 
                 case Errors.NOT_LEADER_FOR_PARTITION =>
-                  info(s"Remote broker is not the leader for partition $topicPartition, which could indicate " +
+                  debug(s"Remote broker is not the leader for partition $topicPartition, which could indicate " +
                     "that the partition is being moved")
                   partitionsWithError += topicPartition
 
                 case _ =>
-                  error(s"Error for partition $topicPartition at offset ${currentPartitionFetchState.fetchOffset}",
+                  error(s"Error for partition $topicPartition at offset ${currentFetchState.fetchOffset}",
                     partitionData.error.exception)
                   partitionsWithError += topicPartition
               }
@@ -318,30 +340,30 @@ abstract class AbstractFetcherThread(name: String,
     partitionMapLock.lockInterruptibly()
     try {
       Option(partitionStates.stateValue(topicPartition)).foreach { state =>
-        val newState = PartitionFetchState(math.min(truncationOffset, state.fetchOffset), state.delay, truncatingLog = true)
+        val newState = PartitionFetchState(math.min(truncationOffset, state.fetchOffset),
+          state.currentLeaderEpoch, state.delay, state = Truncating)
         partitionStates.updateAndMoveToEnd(topicPartition, newState)
+        partitionMapCond.signalAll()
       }
-      partitionMapCond.signalAll()
     } finally partitionMapLock.unlock()
   }
 
-  def addPartitions(initialFetchOffsets: Map[TopicPartition, Long]) {
+  def addPartitions(initialFetchStates: Map[TopicPartition, OffsetAndEpoch]) {
     partitionMapLock.lockInterruptibly()
     try {
-      // If the partitionMap already has the topic/partition, then do not update the map with the old offset
-      val newPartitionToState = initialFetchOffsets.filter { case (tp, _) =>
-        !partitionStates.contains(tp)
-      }.map { case (tp, initialFetchOffset) =>
-        val fetchState =
-          if (initialFetchOffset < 0)
-            new PartitionFetchState(handleOffsetOutOfRange(tp), truncatingLog = true)
+      initialFetchStates.foreach { case (tp, initialFetchState) =>
+        // We can skip the truncation step iff the leader epoch matches the existing epoch
+        val currentState = partitionStates.stateValue(tp)
+        val updatedState = if (currentState != null && currentState.currentLeaderEpoch == initialFetchState.leaderEpoch) {
+          currentState
+        } else {
+          val initialFetchOffset = if (initialFetchState.offset < 0)
+            fetchOffsetAndTruncate(tp, initialFetchState.leaderEpoch)
           else
-            new PartitionFetchState(initialFetchOffset, truncatingLog = true)
-        tp -> fetchState
-      }
-
-      newPartitionToState.foreach { case (tp, state) =>
-        partitionStates.updateAndMoveToEnd(tp, state)
+            initialFetchState.offset
+          PartitionFetchState(initialFetchOffset, initialFetchState.leaderEpoch, state = Truncating)
+        }
+        partitionStates.updateAndMoveToEnd(tp, updatedState)
       }
 
       partitionMapCond.signalAll()
@@ -357,9 +379,13 @@ abstract class AbstractFetcherThread(name: String,
   private def updateFetchOffsetAndMaybeMarkTruncationComplete(fetchOffsets: Map[TopicPartition, OffsetTruncationState]) {
     val newStates: Map[TopicPartition, PartitionFetchState] = partitionStates.partitionStates.asScala
       .map { state =>
+        val currentFetchState = state.value
         val maybeTruncationComplete = fetchOffsets.get(state.topicPartition) match {
-          case Some(offsetTruncationState) => PartitionFetchState(offsetTruncationState.offset, state.value.delay, truncatingLog = !offsetTruncationState.truncationCompleted)
-          case None => state.value()
+          case Some(offsetTruncationState) =>
+            val state = if (offsetTruncationState.truncationCompleted) Fetching else Truncating
+            PartitionFetchState(offsetTruncationState.offset, currentFetchState.currentLeaderEpoch,
+              currentFetchState.delay, state)
+          case None => currentFetchState
         }
         (state.topicPartition, maybeTruncationComplete)
       }.toMap
@@ -389,7 +415,8 @@ abstract class AbstractFetcherThread(name: String,
    * @param tp                    Topic partition
    * @param leaderEpochOffset     Epoch end offset received from the leader for this topic partition
    */
-  private def getOffsetTruncationState(tp: TopicPartition, leaderEpochOffset: EpochEndOffset): OffsetTruncationState = {
+  private def getOffsetTruncationState(tp: TopicPartition,
+                                       leaderEpochOffset: EpochEndOffset): OffsetTruncationState = inLock(partitionMapLock) {
     if (leaderEpochOffset.endOffset == UNDEFINED_EPOCH_OFFSET) {
       // truncate to initial offset which is the high watermark for follower replica. For
       // future replica, it is either high watermark of the future replica or current
@@ -438,9 +465,39 @@ abstract class AbstractFetcherThread(name: String,
   }
 
   /**
+   * Handle the out of range error. Return true if the request succeeded or was fenced, which means we need
+   * not backoff and retry. False if there was a retriable error.
+   */
+  private def handleOutOfRangeError(topicPartition: TopicPartition,
+                                    fetchState: PartitionFetchState): Boolean = {
+    try {
+      val newOffset = fetchOffsetAndTruncate(topicPartition, fetchState.currentLeaderEpoch)
+      val newFetchState = PartitionFetchState(newOffset, fetchState.currentLeaderEpoch, state = Fetching)
+      partitionStates.updateAndMoveToEnd(topicPartition, newFetchState)
+      info(s"Current offset ${fetchState.fetchOffset} for partition $topicPartition is " +
+        s"out of range, which typically implies a leader change. Reset fetch offset to $newOffset")
+      true
+    } catch {
+      case _: FencedLeaderEpochException =>
+        onPartitionFenced(topicPartition)
+        true
+
+      case e @ (_ : UnknownTopicOrPartitionException |
+                _ : UnknownLeaderEpochException |
+                _ : NotLeaderForPartitionException) =>
+        info(s"Could not fetch offset for $topicPartition due to error: ${e.getMessage}")
+        false
+
+      case e: Throwable =>
+        error(s"Error getting offset for partition $topicPartition", e)
+        false
+    }
+  }
+
+  /**
    * Handle a partition whose offset is out of range and return a new fetch offset.
    */
-  protected def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = {
+  protected def fetchOffsetAndTruncate(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = {
     val replicaEndOffset = logEndOffset(topicPartition)
 
     /**
@@ -453,7 +510,7 @@ abstract class AbstractFetcherThread(name: String,
      *
      * There is a potential for a mismatch between the logs of the two replicas here. We don't fix this mismatch as of now.
      */
-    val leaderEndOffset = fetchLatestOffsetFromLeader(topicPartition)
+    val leaderEndOffset = fetchLatestOffsetFromLeader(topicPartition, currentLeaderEpoch)
     if (leaderEndOffset < replicaEndOffset) {
       warn(s"Reset fetch offset for partition $topicPartition from $replicaEndOffset to current " +
         s"leader's latest offset $leaderEndOffset")
@@ -481,7 +538,7 @@ abstract class AbstractFetcherThread(name: String,
        * Putting the two cases together, the follower should fetch from the higher one of its replica log end offset
        * and the current leader's log start offset.
        */
-      val leaderStartOffset = fetchEarliestOffsetFromLeader(topicPartition)
+      val leaderStartOffset = fetchEarliestOffsetFromLeader(topicPartition, currentLeaderEpoch)
       warn(s"Reset fetch offset for partition $topicPartition from $replicaEndOffset to current " +
         s"leader's start offset $leaderStartOffset")
       val offsetToFetch = Math.max(leaderStartOffset, replicaEndOffset)
@@ -492,16 +549,16 @@ abstract class AbstractFetcherThread(name: String,
     }
   }
 
-
   def delayPartitions(partitions: Iterable[TopicPartition], delay: Long) {
     partitionMapLock.lockInterruptibly()
     try {
       for (partition <- partitions) {
-        Option(partitionStates.stateValue(partition)).foreach (currentPartitionFetchState =>
-          if (!currentPartitionFetchState.isDelayed)
-            partitionStates.updateAndMoveToEnd(partition, PartitionFetchState(currentPartitionFetchState.fetchOffset,
-              new DelayedItem(delay), currentPartitionFetchState.truncatingLog))
-        )
+        Option(partitionStates.stateValue(partition)).foreach { currentFetchState =>
+          if (!currentFetchState.isDelayed) {
+            partitionStates.updateAndMoveToEnd(partition, PartitionFetchState(currentFetchState.fetchOffset,
+              currentFetchState.currentLeaderEpoch, new DelayedItem(delay), currentFetchState.state))
+          }
+        }
       }
       partitionMapCond.signalAll()
     } finally partitionMapLock.unlock()
@@ -523,9 +580,17 @@ abstract class AbstractFetcherThread(name: String,
     finally partitionMapLock.unlock()
   }
 
-  private[server] def partitionsAndOffsets: Map[TopicPartition, BrokerAndInitialOffset] = inLock(partitionMapLock) {
+  // Visible for testing
+  private[server] def fetchState(topicPartition: TopicPartition): Option[PartitionFetchState] = inLock(partitionMapLock) {
+    Option(partitionStates.stateValue(topicPartition))
+  }
+
+  private[server] def partitionsAndOffsets: Map[TopicPartition, InitialFetchState] = inLock(partitionMapLock) {
     partitionStates.partitionStates.asScala.map { state =>
-      state.topicPartition -> BrokerAndInitialOffset(sourceBroker, state.value.fetchOffset)
+      val initialFetchState = InitialFetchState(sourceBroker,
+        currentLeaderEpoch = state.value.currentLeaderEpoch,
+        initOffset = state.value.fetchOffset)
+      state.topicPartition -> initialFetchState
     }.toMap
   }
 
@@ -627,28 +692,42 @@ case class ClientIdTopicPartition(clientId: String, topicPartition: TopicPartiti
   override def toString: String = s"$clientId-$topicPartition"
 }
 
-/**
-  * case class to keep partition offset and its state(truncatingLog, delayed)
-  * This represents a partition as being either:
-  * (1) Truncating its log, for example having recently become a follower
-  * (2) Delayed, for example due to an error, where we subsequently back off a bit
-  * (3) ReadyForFetch, the is the active state where the thread is actively fetching data.
-  */
-case class PartitionFetchState(fetchOffset: Long, delay: DelayedItem, truncatingLog: Boolean = false) {
+sealed trait ReplicaState
+case object Truncating extends ReplicaState
+case object Fetching extends ReplicaState
 
-  def this(offset: Long, truncatingLog: Boolean) = this(offset, new DelayedItem(0), truncatingLog)
+object PartitionFetchState {
+  def apply(offset: Long, currentLeaderEpoch: Int, state: ReplicaState): PartitionFetchState = {
+    PartitionFetchState(offset, currentLeaderEpoch, new DelayedItem(0), state)
+  }
+}
 
-  def this(offset: Long, delay: DelayedItem) = this(offset, delay, false)
 
-  def this(fetchOffset: Long) = this(fetchOffset, new DelayedItem(0))
+/**
+ * case class to keep partition offset and its state(truncatingLog, delayed)
+ * This represents a partition as being either:
+ * (1) Truncating its log, for example having recently become a follower
+ * (2) Delayed, for example due to an error, where we subsequently back off a bit
+ * (3) ReadyForFetch, the is the active state where the thread is actively fetching data.
+ */
+case class PartitionFetchState(fetchOffset: Long,
+                               currentLeaderEpoch: Int,
+                               delay: DelayedItem,
+                               state: ReplicaState) {
 
-  def isReadyForFetch: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0 && !truncatingLog
+  def isReadyForFetch: Boolean = state == Fetching && !isDelayed
 
-  def isTruncatingLog: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0 && truncatingLog
+  def isTruncating: Boolean = state == Truncating && !isDelayed
 
   def isDelayed: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) > 0
 
-  override def toString = "offset:%d-isReadyForFetch:%b-isTruncatingLog:%b".format(fetchOffset, isReadyForFetch, truncatingLog)
+  override def toString: String = {
+    s"FetchState(fetchOffset=$fetchOffset" +
+      s", currentLeaderEpoch=$currentLeaderEpoch" +
+      s", state=$state" +
+      s", delay=${delay.delayMs}ms" +
+      s")"
+  }
 }
 
 case class OffsetTruncationState(offset: Long, truncationCompleted: Boolean) {
@@ -658,4 +737,8 @@ case class OffsetTruncationState(offset: Long, truncationCompleted: Boolean) {
   override def toString = "offset:%d-truncationCompleted:%b".format(offset, truncationCompleted)
 }
 
-case class OffsetAndEpoch(offset: Long, epoch: Int)
+case class OffsetAndEpoch(offset: Long, leaderEpoch: Int) {
+  override def toString: String = {
+    s"(offset=$offset, leaderEpoch=$leaderEpoch)"
+  }
+}
diff --git a/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala b/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
index 7a00bc1..a977d9a 100644
--- a/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
+++ b/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
@@ -69,7 +69,7 @@ class DelayedDeleteRecords(delayMs: Long,
   override def tryComplete(): Boolean = {
     // check for each partition if it still has pending acks
     deleteRecordsStatus.foreach { case (topicPartition, status) =>
-      trace(s"Checking delete records satisfaction for ${topicPartition}, current status $status")
+      trace(s"Checking delete records satisfaction for $topicPartition, current status $status")
       // skip those partitions that have already been satisfied
       if (status.acksPending) {
         val (lowWatermarkReached, error, lw) = replicaManager.getPartition(topicPartition) match {
diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala
index 49bea5a..9020099 100644
--- a/core/src/main/scala/kafka/server/DelayedFetch.scala
+++ b/core/src/main/scala/kafka/server/DelayedFetch.scala
@@ -21,9 +21,8 @@ import java.util.concurrent.TimeUnit
 
 import kafka.metrics.KafkaMetricsGroup
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{NotLeaderForPartitionException, UnknownTopicOrPartitionException, KafkaStorageException}
+import org.apache.kafka.common.errors._
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
-import org.apache.kafka.common.requests.IsolationLevel
 
 import scala.collection._
 
@@ -40,17 +39,17 @@ case class FetchMetadata(fetchMinBytes: Int,
                          fetchMaxBytes: Int,
                          hardMaxBytesLimit: Boolean,
                          fetchOnlyLeader: Boolean,
-                         fetchOnlyCommitted: Boolean,
+                         fetchIsolation: FetchIsolation,
                          isFromFollower: Boolean,
                          replicaId: Int,
                          fetchPartitionStatus: Seq[(TopicPartition, FetchPartitionStatus)]) {
 
-  override def toString = "[minBytes: " + fetchMinBytes + ", " +
-    "maxBytes:" + fetchMaxBytes + ", " +
-    "onlyLeader:" + fetchOnlyLeader + ", " +
-    "onlyCommitted: " + fetchOnlyCommitted + ", " +
-    "replicaId: " + replicaId + ", " +
-    "partitionStatus: " + fetchPartitionStatus + "]"
+  override def toString = "FetchMetadata(minBytes=" + fetchMinBytes + ", " +
+    "maxBytes=" + fetchMaxBytes + ", " +
+    "onlyLeader=" + fetchOnlyLeader + ", " +
+    "fetchIsolation=" + fetchIsolation + ", " +
+    "replicaId=" + replicaId + ", " +
+    "partitionStatus=" + fetchPartitionStatus + ")"
 }
 /**
  * A delayed fetch operation that can be created by the replica manager and watched
@@ -60,7 +59,6 @@ class DelayedFetch(delayMs: Long,
                    fetchMetadata: FetchMetadata,
                    replicaManager: ReplicaManager,
                    quota: ReplicaQuota,
-                   isolationLevel: IsolationLevel,
                    responseCallback: Seq[(TopicPartition, FetchPartitionData)] => Unit)
   extends DelayedOperation(delayMs) {
 
@@ -72,24 +70,27 @@ class DelayedFetch(delayMs: Long,
    * Case C: The fetch offset locates not on the last segment of the log
    * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes
    * Case E: The partition is in an offline log directory on this broker
+   * Case F: This broker is the leader, but the requested epoch is now fenced
    *
    * Upon completion, should return whatever data is available for each valid partition
    */
-  override def tryComplete() : Boolean = {
+  override def tryComplete(): Boolean = {
     var accumulatedSize = 0
     fetchMetadata.fetchPartitionStatus.foreach {
       case (topicPartition, fetchStatus) =>
         val fetchOffset = fetchStatus.startOffsetMetadata
+        val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
         try {
           if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
-            val replica = replicaManager.getLeaderReplicaIfLocal(topicPartition)
-            val endOffset =
-              if (isolationLevel == IsolationLevel.READ_COMMITTED)
-                replica.lastStableOffset
-              else if (fetchMetadata.fetchOnlyCommitted)
-                replica.highWatermark
-              else
-                replica.logEndOffset
+            val partition = replicaManager.getPartitionOrException(topicPartition,
+              expectLeader = fetchMetadata.fetchOnlyLeader)
+            val offsetSnapshot = partition.fetchOffsetSnapshot(fetchLeaderEpoch, fetchMetadata.fetchOnlyLeader)
+
+            val endOffset = fetchMetadata.fetchIsolation match {
+              case FetchLogEnd => offsetSnapshot.logEndOffset
+              case FetchHighWatermark => offsetSnapshot.highWatermark
+              case FetchTxnCommitted => offsetSnapshot.lastStableOffset
+            }
 
             // Go directly to the check for Case D if the message offsets are the same. If the log segment
             // has just rolled, then the high watermark offset will remain the same but be on the old segment,
@@ -97,12 +98,12 @@ class DelayedFetch(delayMs: Long,
             if (endOffset.messageOffset != fetchOffset.messageOffset) {
               if (endOffset.onOlderSegment(fetchOffset)) {
                 // Case C, this can happen when the new fetch operation is on a truncated leader
-                debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicPartition))
+                debug(s"Satisfying fetch $fetchMetadata since it is fetching later segments of partition $topicPartition.")
                 return forceComplete()
               } else if (fetchOffset.onOlderSegment(endOffset)) {
                 // Case C, this can happen when the fetch operation is falling behind the current segment
                 // or the partition has just rolled a new segment
-                debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata))
+                debug(s"Satisfying fetch $fetchMetadata immediately since it is fetching older segments.")
                 // We will not force complete the fetch request if a replica should be throttled.
                 if (!replicaManager.shouldLeaderThrottle(quota, topicPartition, fetchMetadata.replicaId))
                   return forceComplete()
@@ -116,10 +117,14 @@ class DelayedFetch(delayMs: Long,
           }
         } catch {
           case _: KafkaStorageException => // Case E
-            debug("Partition %s is in an offline log directory, satisfy %s immediately".format(topicPartition, fetchMetadata))
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchMetadata immediately")
             return forceComplete()
           case _: UnknownTopicOrPartitionException => // Case B
-            debug("Broker no longer know of %s, satisfy %s immediately".format(topicPartition, fetchMetadata))
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchMetadata immediately")
+            return forceComplete()
+          case _: FencedLeaderEpochException => // Case F
+            debug(s"Broker is the leader of partition $topicPartition, but the requested epoch " +
+              s"$fetchLeaderEpoch is fenced by the latest leader epoch, satisfy $fetchMetadata immediately")
             return forceComplete()
           case _: NotLeaderForPartitionException =>  // Case A
             debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicPartition, fetchMetadata))
@@ -148,12 +153,11 @@ class DelayedFetch(delayMs: Long,
     val logReadResults = replicaManager.readFromLocalLog(
       replicaId = fetchMetadata.replicaId,
       fetchOnlyFromLeader = fetchMetadata.fetchOnlyLeader,
-      readOnlyCommitted = fetchMetadata.fetchOnlyCommitted,
+      fetchIsolation = fetchMetadata.fetchIsolation,
       fetchMaxBytes = fetchMetadata.fetchMaxBytes,
       hardMaxBytesLimit = fetchMetadata.hardMaxBytesLimit,
       readPartitionInfo = fetchMetadata.fetchPartitionStatus.map { case (tp, status) => tp -> status.fetchInfo },
-      quota = quota,
-      isolationLevel = isolationLevel)
+      quota = quota)
 
     val fetchPartitionData = logReadResults.map { case (tp, result) =>
       tp -> FetchPartitionData(result.error, result.highWatermark, result.leaderLogStartOffset, result.info.records,
diff --git a/core/src/main/scala/kafka/server/FetchDataInfo.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala
index cbd54c0..a55fe23 100644
--- a/core/src/main/scala/kafka/server/FetchDataInfo.scala
+++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala
@@ -20,6 +20,11 @@ package kafka.server
 import org.apache.kafka.common.record.Records
 import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
 
+sealed trait FetchIsolation
+case object FetchLogEnd extends FetchIsolation
+case object FetchHighWatermark extends FetchIsolation
+case object FetchTxnCommitted extends FetchIsolation
+
 case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
                          records: Records,
                          firstEntryIncomplete: Boolean = false,
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index d0be97f..18ab9db 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -31,7 +31,6 @@ import kafka.common.OffsetAndMetadata
 import kafka.controller.KafkaController
 import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult}
 import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
-import kafka.log.{Log, LogManager, TimestampOffset}
 import kafka.network.RequestChannel
 import kafka.security.SecurityUtils
 import kafka.security.auth.{Resource, _}
@@ -494,10 +493,11 @@ class KafkaApis(val requestChannel: RequestChannel,
     val versionId = request.header.apiVersion
     val clientId = request.header.clientId
     val fetchRequest = request.body[FetchRequest]
-    val fetchContext = fetchManager.newContext(fetchRequest.metadata(),
-          fetchRequest.fetchData(),
-          fetchRequest.toForget(),
-          fetchRequest.isFromFollower())
+    val fetchContext = fetchManager.newContext(
+      fetchRequest.metadata,
+      fetchRequest.fetchData,
+      fetchRequest.toForget,
+      fetchRequest.isFromFollower)
 
     def errorResponse[T >: MemoryRecords <: BaseRecords](error: Errors): FetchResponse.PartitionData[T] = {
       new FetchResponse.PartitionData[T](error, FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET,
@@ -506,7 +506,7 @@ class KafkaApis(val requestChannel: RequestChannel,
 
     val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponse.PartitionData[Records])]()
     val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    if (fetchRequest.isFromFollower()) {
+    if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authorize(request.session, ClusterAction, Resource.ClusterResource)) {
         fetchContext.foreachPartition { (topicPartition, data) =>
@@ -656,7 +656,7 @@ class KafkaApis(val requestChannel: RequestChannel,
         } else {
           // Get the actual response. This will update the fetch context.
           unconvertedFetchResponse = fetchContext.updateAndGenerateResponseData(partitions)
-          trace(s"Sending Fetch response with partitions.size=${responseSize}, metadata=${unconvertedFetchResponse.sessionId()}")
+          trace(s"Sending Fetch response with partitions.size=$responseSize, metadata=${unconvertedFetchResponse.sessionId}")
         }
 
         // Send the response immediately.
@@ -746,26 +746,12 @@ class KafkaApis(val requestChannel: RequestChannel,
 
     val responseMap = authorizedRequestInfo.map {case (topicPartition, partitionData) =>
       try {
-        // ensure leader exists
-        val localReplica = if (offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID)
-          replicaManager.getLeaderReplicaIfLocal(topicPartition)
-        else
-          replicaManager.getReplicaOrException(topicPartition)
-        val offsets = {
-          val allOffsets = fetchOffsets(replicaManager.logManager,
-            topicPartition,
-            partitionData.timestamp,
-            partitionData.maxNumOffsets)
-          if (offsetRequest.replicaId != ListOffsetRequest.CONSUMER_REPLICA_ID) {
-            allOffsets
-          } else {
-            val hw = localReplica.highWatermark.messageOffset
-            if (allOffsets.exists(_ > hw))
-              hw +: allOffsets.dropWhile(_ > hw)
-            else
-              allOffsets
-          }
-        }
+        val offsets = replicaManager.legacyFetchOffsetsForTimestamp(
+          topicPartition = topicPartition,
+          timestamp = partitionData.timestamp,
+          maxNumOffsets = partitionData.maxNumOffsets,
+          isFromConsumer = offsetRequest.replicaId == ListOffsetRequest.CONSUMER_REPLICA_ID,
+          fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID)
         (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE, offsets.map(new JLong(_)).asJava))
       } catch {
         // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages
@@ -801,7 +787,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     })
 
     val responseMap = authorizedRequestInfo.map { case (topicPartition, partitionData) =>
-      if (offsetRequest.duplicatePartitions().contains(topicPartition)) {
+      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,
@@ -810,32 +796,17 @@ class KafkaApis(val requestChannel: RequestChannel,
           Optional.empty()))
       } else {
         try {
-          // ensure leader exists
-          val localReplica = if (offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID)
-            replicaManager.getLeaderReplicaIfLocal(topicPartition)
+          val fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID
+          val isolationLevelOpt = if (offsetRequest.replicaId == ListOffsetRequest.CONSUMER_REPLICA_ID)
+            Some(offsetRequest.isolationLevel)
           else
-            replicaManager.getReplicaOrException(topicPartition)
-
-          val fromConsumer = offsetRequest.replicaId == ListOffsetRequest.CONSUMER_REPLICA_ID
-          val found = if (fromConsumer) {
-            val lastFetchableOffset = offsetRequest.isolationLevel match {
-              case IsolationLevel.READ_COMMITTED => localReplica.lastStableOffset.messageOffset
-              case IsolationLevel.READ_UNCOMMITTED => localReplica.highWatermark.messageOffset
-            }
-
-            if (partitionData.timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
-              TimestampOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset)
-            else {
-              def allowed(timestampOffset: TimestampOffset): Boolean =
-                partitionData.timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP || timestampOffset.offset < lastFetchableOffset
+            None
 
-              fetchOffsetForTimestamp(topicPartition, partitionData.timestamp)
-                .filter(allowed).getOrElse(TimestampOffset.Unknown)
-            }
-          } else {
-            fetchOffsetForTimestamp(topicPartition, partitionData.timestamp)
-              .getOrElse(TimestampOffset.Unknown)
-          }
+          val found = replicaManager.fetchOffsetForTimestamp(topicPartition,
+            partitionData.timestamp,
+            isolationLevelOpt,
+            partitionData.currentLeaderEpoch,
+            fetchOnlyFromLeader)
 
           (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE, found.timestamp, found.offset,
             Optional.empty()))
@@ -844,6 +815,8 @@ class KafkaApis(val requestChannel: RequestChannel,
           // would have received a clear exception and there is no value in logging the entire stack trace for the same
           case e @ (_ : UnknownTopicOrPartitionException |
                     _ : NotLeaderForPartitionException |
+                    _ : UnknownLeaderEpochException |
+                    _ : FencedLeaderEpochException |
                     _ : KafkaStorageException |
                     _ : UnsupportedForMessageFormatException) =>
             debug(s"Offset request with correlation id $correlationId from client $clientId on " +
@@ -864,72 +837,6 @@ class KafkaApis(val requestChannel: RequestChannel,
     responseMap ++ unauthorizedResponseStatus
   }
 
-  def fetchOffsets(logManager: LogManager, topicPartition: TopicPartition, timestamp: Long, maxNumOffsets: Int): Seq[Long] = {
-    logManager.getLog(topicPartition) match {
-      case Some(log) =>
-        fetchOffsetsBefore(log, timestamp, maxNumOffsets)
-      case None =>
-        if (timestamp == ListOffsetRequest.LATEST_TIMESTAMP || timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
-          Seq(0L)
-        else
-          Nil
-    }
-  }
-
-  private def fetchOffsetForTimestamp(topicPartition: TopicPartition, timestamp: Long): Option[TimestampOffset] = {
-    replicaManager.getLog(topicPartition) match {
-      case Some(log) =>
-        log.fetchOffsetsByTimestamp(timestamp)
-      case None =>
-        throw new UnknownTopicOrPartitionException(s"$topicPartition does not exist on the broker.")
-    }
-  }
-
-  private[server] def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = {
-    // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides
-    // constant time access while being safe to use with concurrent collections unlike `toArray`.
-    val segments = log.logSegments.toBuffer
-    val lastSegmentHasSize = segments.last.size > 0
-
-    val offsetTimeArray =
-      if (lastSegmentHasSize)
-        new Array[(Long, Long)](segments.length + 1)
-      else
-        new Array[(Long, Long)](segments.length)
-
-    for (i <- segments.indices)
-      offsetTimeArray(i) = (math.max(segments(i).baseOffset, log.logStartOffset), segments(i).lastModified)
-    if (lastSegmentHasSize)
-      offsetTimeArray(segments.length) = (log.logEndOffset, time.milliseconds)
-
-    var startIndex = -1
-    timestamp match {
-      case ListOffsetRequest.LATEST_TIMESTAMP =>
-        startIndex = offsetTimeArray.length - 1
-      case ListOffsetRequest.EARLIEST_TIMESTAMP =>
-        startIndex = 0
-      case _ =>
-        var isFound = false
-        debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2)))
-        startIndex = offsetTimeArray.length - 1
-        while (startIndex >= 0 && !isFound) {
-          if (offsetTimeArray(startIndex)._2 <= timestamp)
-            isFound = true
-          else
-            startIndex -= 1
-        }
-    }
-
-    val retSize = maxNumOffsets.min(startIndex + 1)
-    val ret = new Array[Long](retSize)
-    for (j <- 0 until retSize) {
-      ret(j) = offsetTimeArray(startIndex)._1
-      startIndex -= 1
-    }
-    // ensure that the returned seq is in descending order of offsets
-    ret.toSeq.sortBy(-_)
-  }
-
   private def createTopic(topic: String,
                           numPartitions: Int,
                           replicationFactor: Int,
diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala
index bded125..1616b84 100644
--- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala
@@ -17,19 +17,21 @@
 
 package kafka.server
 
-
 import kafka.cluster.BrokerEndPoint
 
 class ReplicaAlterLogDirsManager(brokerConfig: KafkaConfig,
                                  replicaManager: ReplicaManager,
                                  quotaManager: ReplicationQuotaManager,
                                  brokerTopicStats: BrokerTopicStats)
-  extends AbstractFetcherManager(s"ReplicaAlterLogDirsManager on broker ${brokerConfig.brokerId}",
-    "ReplicaAlterLogDirs", brokerConfig.getNumReplicaAlterLogDirsThreads) {
+  extends AbstractFetcherManager[ReplicaAlterLogDirsThread](
+    name = s"ReplicaAlterLogDirsManager on broker ${brokerConfig.brokerId}",
+    clientId = "ReplicaAlterLogDirs",
+    numFetchers = brokerConfig.getNumReplicaAlterLogDirsThreads) {
 
-  override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = {
+  override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): ReplicaAlterLogDirsThread = {
     val threadName = s"ReplicaAlterLogDirsThread-$fetcherId"
-    new ReplicaAlterLogDirsThread(threadName, sourceBroker, brokerConfig, replicaManager, quotaManager, brokerTopicStats)
+    new ReplicaAlterLogDirsThread(threadName, sourceBroker, brokerConfig, replicaManager,
+      quotaManager, brokerTopicStats)
   }
 
   def shutdown() {
diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
index 2244771..7761f35 100644
--- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
@@ -22,7 +22,7 @@ import java.util.Optional
 
 import kafka.api.Request
 import kafka.cluster.BrokerEndPoint
-import kafka.log.LogAppendInfo
+import kafka.log.{LogAppendInfo, LogOffsetSnapshot}
 import kafka.server.AbstractFetcherThread.ResultWithPartitions
 import kafka.server.QuotaFactory.UnboundedQuota
 import org.apache.kafka.common.TopicPartition
@@ -53,15 +53,15 @@ class ReplicaAlterLogDirsThread(name: String,
   private val fetchSize = brokerConfig.replicaFetchMaxBytes
 
   override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = {
-    replicaMgr.getReplicaOrException(topicPartition, Request.FutureLocalReplicaId).epochs.map(_.latestEpoch)
+    replicaMgr.futureLocalReplicaOrException(topicPartition).epochs.map(_.latestEpoch)
   }
 
   override protected def logEndOffset(topicPartition: TopicPartition): Long = {
-    replicaMgr.getReplicaOrException(topicPartition, Request.FutureLocalReplicaId).logEndOffset.messageOffset
+    replicaMgr.futureLocalReplicaOrException(topicPartition).logEndOffset.messageOffset
   }
 
   override protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch] = {
-    val replica = replicaMgr.getReplicaOrException(topicPartition, Request.FutureLocalReplicaId)
+    val replica = replicaMgr.futureLocalReplicaOrException(topicPartition)
     replica.epochs.flatMap { epochCache =>
       val (foundEpoch, foundOffset) = epochCache.endOffsetFor(epoch)
       if (foundOffset == UNDEFINED_EPOCH_OFFSET)
@@ -71,7 +71,7 @@ class ReplicaAlterLogDirsThread(name: String,
     }
   }
 
-  def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, PD)] = {
+  def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, FetchData)] = {
     var partitionData: Seq[(TopicPartition, FetchResponse.PartitionData[Records])] = null
     val request = fetchRequest.build()
 
@@ -105,7 +105,7 @@ class ReplicaAlterLogDirsThread(name: String,
   override def processPartitionData(topicPartition: TopicPartition,
                                     fetchOffset: Long,
                                     partitionData: PartitionData[Records]): Option[LogAppendInfo] = {
-    val futureReplica = replicaMgr.getReplicaOrException(topicPartition, Request.FutureLocalReplicaId)
+    val futureReplica = replicaMgr.futureLocalReplicaOrException(topicPartition)
     val partition = replicaMgr.getPartition(topicPartition).get
     val records = toMemoryRecords(partitionData.records)
 
@@ -125,12 +125,19 @@ class ReplicaAlterLogDirsThread(name: String,
     logAppendInfo
   }
 
-  override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition): Long = {
-    replicaMgr.getReplicaOrException(topicPartition).logStartOffset
+  override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, leaderEpoch: Int): Long = {
+    val offsetSnapshot = offsetSnapshotFromCurrentReplica(topicPartition, leaderEpoch)
+    offsetSnapshot.logStartOffset
   }
 
-  override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition): Long = {
-    replicaMgr.getReplicaOrException(topicPartition).logEndOffset.messageOffset
+  override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, leaderEpoch: Int): Long = {
+    val offsetSnapshot = offsetSnapshotFromCurrentReplica(topicPartition, leaderEpoch)
+    offsetSnapshot.logEndOffset.messageOffset
+  }
+
+  private def offsetSnapshotFromCurrentReplica(topicPartition: TopicPartition, leaderEpoch: Int): LogOffsetSnapshot = {
+    val partition = replicaMgr.getPartitionOrException(topicPartition, expectLeader = false)
+    partition.fetchOffsetSnapshot(Optional.of[Integer](leaderEpoch), fetchOnlyFromLeader = false)
   }
 
   /**
@@ -138,11 +145,19 @@ class ReplicaAlterLogDirsThread(name: String,
    * @param partitions map of topic partition -> leader epoch of the future replica
    * @return map of topic partition -> end offset for a requested leader epoch
    */
-  override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = {
-    partitions.map { case (tp, epoch) =>
+  override def fetchEpochsFromLeader(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = {
+    partitions.map { case (tp, epochData) =>
       try {
-        val (leaderEpoch, leaderOffset) = replicaMgr.getReplicaOrException(tp).epochs.get.endOffsetFor(epoch)
-        tp -> new EpochEndOffset(Errors.NONE, leaderEpoch, leaderOffset)
+        val endOffset = if (epochData.leaderEpoch == UNDEFINED_EPOCH) {
+          new EpochEndOffset(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
+        } else {
+          val partition = replicaMgr.getPartitionOrException(tp, expectLeader = false)
+          partition.lastOffsetForLeaderEpoch(
+            currentLeaderEpoch = epochData.currentLeaderEpoch,
+            leaderEpoch = epochData.leaderEpoch,
+            fetchOnlyFromLeader = false)
+        }
+        tp -> endOffset
       } catch {
         case t: Throwable =>
           warn(s"Error when getting EpochEndOffset for $tp", t)
@@ -166,12 +181,12 @@ class ReplicaAlterLogDirsThread(name: String,
    * exchange with the current replica to truncate to the largest common log prefix for the topic partition
    */
   override def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = {
-    val partition = replicaMgr.getPartition(topicPartition).get
+    val partition = replicaMgr.getPartitionOrException(topicPartition, expectLeader = false)
     partition.truncateTo(truncationState.offset, isFuture = true)
   }
 
   override protected def truncateFullyAndStartAt(topicPartition: TopicPartition, offset: Long): Unit = {
-    val partition = replicaMgr.getPartition(topicPartition).get
+    val partition = replicaMgr.getPartitionOrException(topicPartition, expectLeader = false)
     partition.truncateFullyAndStartAt(offset, isFuture = true)
   }
 
@@ -194,9 +209,9 @@ class ReplicaAlterLogDirsThread(name: String,
     if (maxPartitionOpt.nonEmpty) {
       val (topicPartition, partitionFetchState) = maxPartitionOpt.get
       try {
-        val logStartOffset = replicaMgr.getReplicaOrException(topicPartition, Request.FutureLocalReplicaId).logStartOffset
+        val logStartOffset = replicaMgr.futureLocalReplicaOrException(topicPartition).logStartOffset
         requestMap.put(topicPartition, new FetchRequest.PartitionData(partitionFetchState.fetchOffset, logStartOffset,
-          fetchSize, Optional.empty()))
+          fetchSize, Optional.of(partitionFetchState.currentLeaderEpoch)))
       } catch {
         case _: KafkaStorageException =>
           partitionsWithError += topicPartition
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala
index a50b0bb..fa902b9 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala
@@ -21,15 +21,22 @@ import kafka.cluster.BrokerEndPoint
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.utils.Time
 
-class ReplicaFetcherManager(brokerConfig: KafkaConfig, protected val replicaManager: ReplicaManager, metrics: Metrics,
-                            time: Time, threadNamePrefix: Option[String] = None, quotaManager: ReplicationQuotaManager)
-      extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId,
-        "Replica", brokerConfig.numReplicaFetchers) {
+class ReplicaFetcherManager(brokerConfig: KafkaConfig,
+                            protected val replicaManager: ReplicaManager,
+                            metrics: Metrics,
+                            time: Time,
+                            threadNamePrefix: Option[String] = None,
+                            quotaManager: ReplicationQuotaManager)
+      extends AbstractFetcherManager[ReplicaFetcherThread](
+        name = "ReplicaFetcherManager on broker " + brokerConfig.brokerId,
+        clientId = "Replica",
+        numFetchers = brokerConfig.numReplicaFetchers) {
 
-  override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = {
-    val prefix = threadNamePrefix.map(tp => s"${tp}:").getOrElse("")
+  override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): ReplicaFetcherThread = {
+    val prefix = threadNamePrefix.map(tp => s"$tp:").getOrElse("")
     val threadName = s"${prefix}ReplicaFetcherThread-$fetcherId-${sourceBroker.id}"
-    new ReplicaFetcherThread(threadName, fetcherId, sourceBroker, brokerConfig, replicaManager, metrics, time, quotaManager)
+    new ReplicaFetcherThread(threadName, fetcherId, sourceBroker, brokerConfig, replicaManager,
+      metrics, time, quotaManager)
   }
 
   def shutdown() {
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index bdbadd9..930281a 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -94,15 +94,15 @@ class ReplicaFetcherThread(name: String,
   private val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id)
 
   override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = {
-    replicaMgr.getReplicaOrException(topicPartition).epochs.map(_.latestEpoch)
+    replicaMgr.localReplicaOrException(topicPartition).epochs.map(_.latestEpoch)
   }
 
   override protected def logEndOffset(topicPartition: TopicPartition): Long = {
-    replicaMgr.getReplicaOrException(topicPartition).logEndOffset.messageOffset
+    replicaMgr.localReplicaOrException(topicPartition).logEndOffset.messageOffset
   }
 
   override protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch] = {
-    val replica = replicaMgr.getReplicaOrException(topicPartition)
+    val replica = replicaMgr.localReplicaOrException(topicPartition)
     replica.epochs.flatMap { epochCache =>
       val (foundEpoch, foundOffset) = epochCache.endOffsetFor(epoch)
       if (foundOffset == UNDEFINED_EPOCH_OFFSET)
@@ -123,8 +123,8 @@ class ReplicaFetcherThread(name: String,
   // process fetched data
   override def processPartitionData(topicPartition: TopicPartition,
                                     fetchOffset: Long,
-                                    partitionData: PD): Option[LogAppendInfo] = {
-    val replica = replicaMgr.getReplicaOrException(topicPartition)
+                                    partitionData: FetchData): Option[LogAppendInfo] = {
+    val replica = replicaMgr.localReplicaOrException(topicPartition)
     val partition = replicaMgr.getPartition(topicPartition).get
     val records = toMemoryRecords(partitionData.records)
 
@@ -172,7 +172,8 @@ class ReplicaFetcherThread(name: String,
         "equal or larger than your settings for max.message.bytes, both at a broker and topic level.")
   }
 
-  override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, PD)] = {
+
+  override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, FetchData)] = {
     try {
       val clientResponse = leaderEndpoint.sendRequest(fetchRequest)
       val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse[Records]]
@@ -188,16 +189,17 @@ class ReplicaFetcherThread(name: String,
     }
   }
 
-  override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition): Long = {
-    fetchOffsetFromLeader(topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP)
+  override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = {
+    fetchOffsetFromLeader(topicPartition, currentLeaderEpoch, ListOffsetRequest.EARLIEST_TIMESTAMP)
   }
 
-  override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition): Long = {
-    fetchOffsetFromLeader(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP)
+  override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = {
+    fetchOffsetFromLeader(topicPartition, currentLeaderEpoch, ListOffsetRequest.LATEST_TIMESTAMP)
   }
 
-  private def fetchOffsetFromLeader(topicPartition: TopicPartition, earliestOrLatest: Long): Long = {
-    val requestPartitionData = new ListOffsetRequest.PartitionData(earliestOrLatest, Optional.empty[Integer]())
+  private def fetchOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int, earliestOrLatest: Long): Long = {
+    val requestPartitionData = new ListOffsetRequest.PartitionData(earliestOrLatest,
+      Optional.of[Integer](currentLeaderEpoch))
     val requestPartitions = Map(topicPartition -> requestPartitionData)
     val requestBuilder = ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId)
       .setTargetTimes(requestPartitions.asJava)
@@ -220,13 +222,13 @@ class ReplicaFetcherThread(name: String,
     val partitionsWithError = mutable.Set[TopicPartition]()
 
     val builder = fetchSessionHandler.newBuilder()
-    partitionMap.foreach { case (topicPartition, partitionFetchState) =>
+    partitionMap.foreach { case (topicPartition, fetchState) =>
       // We will not include a replica in the fetch request if it should be throttled.
-      if (partitionFetchState.isReadyForFetch && !shouldFollowerThrottle(quota, topicPartition)) {
+      if (fetchState.isReadyForFetch && !shouldFollowerThrottle(quota, topicPartition)) {
         try {
-          val logStartOffset = replicaMgr.getReplicaOrException(topicPartition).logStartOffset
+          val logStartOffset = replicaMgr.localReplicaOrException(topicPartition).logStartOffset
           builder.add(topicPartition, new FetchRequest.PartitionData(
-            partitionFetchState.fetchOffset, logStartOffset, fetchSize, Optional.empty()))
+            fetchState.fetchOffset, logStartOffset, fetchSize, Optional.of(fetchState.currentLeaderEpoch)))
         } catch {
           case _: KafkaStorageException =>
             // The replica has already been marked offline due to log directory failure and the original failure should have already been logged.
@@ -256,7 +258,7 @@ class ReplicaFetcherThread(name: String,
    * The logic for finding the truncation offset is implemented in AbstractFetcherThread.getOffsetTruncationState
    */
   override def truncate(tp: TopicPartition, offsetTruncationState: OffsetTruncationState): Unit = {
-    val replica = replicaMgr.getReplicaOrException(tp)
+    val replica = replicaMgr.localReplicaOrException(tp)
     val partition = replicaMgr.getPartition(tp).get
     partition.truncateTo(offsetTruncationState.offset, isFuture = false)
 
@@ -266,7 +268,8 @@ class ReplicaFetcherThread(name: String,
 
     // mark the future replica for truncation only when we do last truncation
     if (offsetTruncationState.truncationCompleted)
-      replicaMgr.replicaAlterLogDirsManager.markPartitionsForTruncation(brokerConfig.brokerId, tp, offsetTruncationState.offset)
+      replicaMgr.replicaAlterLogDirsManager.markPartitionsForTruncation(brokerConfig.brokerId, tp,
+        offsetTruncationState.offset)
   }
 
   override protected def truncateFullyAndStartAt(topicPartition: TopicPartition, offset: Long): Unit = {
@@ -274,44 +277,46 @@ class ReplicaFetcherThread(name: String,
     partition.truncateFullyAndStartAt(offset, isFuture = false)
   }
 
-  override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = {
-    var result: Map[TopicPartition, EpochEndOffset] = null
+  override def fetchEpochsFromLeader(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = {
+    def undefinedResponseMap(error: Errors,
+                             requestMap: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = {
+      requestMap.map { case (tp, _) =>
+        tp -> new EpochEndOffset(error, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
+      }
+    }
+
     if (brokerSupportsLeaderEpochRequest) {
       // skip request for partitions without epoch, as their topic log message format doesn't support epochs
-      val (partitionsWithEpoch, partitionsWithoutEpoch) = partitions.partition { case (_, epoch) => epoch != UNDEFINED_EPOCH }
-      val resultWithoutEpoch = partitionsWithoutEpoch.map { case (tp, _) => (tp, new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)) }
-
+      val (partitionsWithEpoch, partitionsWithoutEpoch) = partitions.partition { case (_, epochData) =>
+        epochData.leaderEpoch != UNDEFINED_EPOCH
+      }
+      val resultWithoutEpoch = undefinedResponseMap(Errors.NONE, partitionsWithoutEpoch)
       if (partitionsWithEpoch.isEmpty) {
         debug("Skipping leaderEpoch request since all partitions do not have an epoch")
         return resultWithoutEpoch
       }
 
-      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 epochRequest = new OffsetsForLeaderEpochRequest.Builder(offsetForLeaderEpochRequestVersion,
+        partitionsWithEpoch.asJava)
+      val resultWithEpoch = try {
         val response = leaderEndpoint.sendRequest(epochRequest)
-
-        result = response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse].responses.asScala ++ resultWithoutEpoch
-        debug(s"Receive leaderEpoch response $result; Skipped request for partitions ${partitionsWithoutEpoch.keys}")
+        val responseBody = response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse]
+        debug(s"Receive leaderEpoch response $response; " +
+          s"Skipped request for partitions ${partitionsWithoutEpoch.keys}")
+        responseBody.responses.asScala
       } catch {
         case t: Throwable =>
           warn(s"Error when sending leader epoch request for $partitions", t)
 
           // if we get any unexpected exception, mark all partitions with an error
-          result = partitions.map { case (tp, _) =>
-            tp -> new EpochEndOffset(Errors.forException(t), UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
-          }
+          undefinedResponseMap(Errors.forException(t), partitionsWithEpoch)
       }
+      resultWithEpoch ++ resultWithoutEpoch
     } else {
       // just generate a response with no error but UNDEFINED_OFFSET so that we can fall back to truncating using
       // high watermark in maybeTruncate()
-      result = partitions.map { case (tp, _) =>
-        tp -> new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
-      }
+      undefinedResponseMap(Errors.NONE, partitions)
     }
-    result
   }
 
   /**
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 114e69c..d5a3c68 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -17,6 +17,7 @@
 package kafka.server
 
 import java.io.File
+import java.util.Optional
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
 import java.util.concurrent.locks.Lock
@@ -25,7 +26,7 @@ import com.yammer.metrics.core.Gauge
 import kafka.api._
 import kafka.cluster.{BrokerEndPoint, Partition, Replica}
 import kafka.controller.{KafkaController, StateChangeLogger}
-import kafka.log.{Log, LogAppendInfo, LogConfig, LogManager}
+import kafka.log._
 import kafka.metrics.KafkaMetricsGroup
 import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
 import kafka.server.checkpoints.OffsetCheckpointFile
@@ -120,7 +121,14 @@ object ReplicaManager {
   val HighWatermarkFilename = "replication-offset-checkpoint"
   val IsrChangePropagationBlackOut = 5000L
   val IsrChangePropagationInterval = 60000L
-  val OfflinePartition = new Partition("", -1, null, null, isOffline = true)
+  val OfflinePartition: Partition = new Partition(new TopicPartition("", -1),
+    isOffline = true,
+    replicaLagTimeMaxMs = 0L,
+    localBrokerId = -1,
+    time = null,
+    replicaManager = null,
+    logManager = null,
+    zkClient = null)
 }
 
 class ReplicaManager(val config: KafkaConfig,
@@ -169,7 +177,7 @@ class ReplicaManager(val config: KafkaConfig,
   @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch
   private val localBrokerId = config.brokerId
   private val allPartitions = new Pool[TopicPartition, Partition](valueFactory = Some(tp =>
-    new Partition(tp.topic, tp.partition, time, this)))
+    Partition(tp, time, this)))
   private val replicaStateChangeLock = new Object
   val replicaFetcherManager = createReplicaFetcherManager(metrics, time, threadNamePrefix, quotaManagers.follower)
   val replicaAlterLogDirsManager = createReplicaAlterLogDirsManager(quotaManagers.alterLogDirs, brokerTopicStats)
@@ -396,66 +404,49 @@ class ReplicaManager(val config: KafkaConfig,
   private def offlinePartitionsIterator: Iterator[Partition] =
     allPartitions.values.iterator.filter(_ eq ReplicaManager.OfflinePartition)
 
-  def getReplicaOrException(topicPartition: TopicPartition, brokerId: Int): Replica = {
+
+  def getPartitionOrException(topicPartition: TopicPartition, expectLeader: Boolean): Partition = {
     getPartition(topicPartition) match {
       case Some(partition) =>
         if (partition eq ReplicaManager.OfflinePartition)
-          throw new KafkaStorageException(s"Replica $brokerId is in an offline log directory for partition $topicPartition")
+          throw new KafkaStorageException(s"Partition $topicPartition is in an offline log directory")
         else
-          partition.getReplica(brokerId).getOrElse(
-            throw new ReplicaNotAvailableException(s"Replica $brokerId is not available for partition $topicPartition"))
+          partition
 
       case None if metadataCache.contains(topicPartition) =>
-        throw new ReplicaNotAvailableException(s"Replica $brokerId is not available for partition $topicPartition")
+        if (expectLeader) {
+          // The topic exists, but this broker is no longer a replica of it, so we return NOT_LEADER which
+          // forces clients to refresh metadata to find the new location. This can happen, for example,
+          // during a partition reassignment if a produce request from the client is sent to a broker after
+          // the local replica has been deleted.
+          throw new NotLeaderForPartitionException(s"Broker $localBrokerId is not a replica of $topicPartition")
+        } else {
+          throw new ReplicaNotAvailableException(s"Partition $topicPartition is not available")
+        }
 
       case None =>
         throw new UnknownTopicOrPartitionException(s"Partition $topicPartition doesn't exist")
     }
   }
 
-  def getReplicaOrException(topicPartition: TopicPartition): Replica = {
-    getReplicaOrException(topicPartition, localBrokerId)
+  def localReplicaOrException(topicPartition: TopicPartition): Replica = {
+    getPartitionOrException(topicPartition, expectLeader = false).localReplicaOrException
   }
 
-  def getLeaderReplicaIfLocal(topicPartition: TopicPartition): Replica =  {
-    val (_, replica) = getPartitionAndLeaderReplicaIfLocal(topicPartition)
-    replica
+  def futureLocalReplicaOrException(topicPartition: TopicPartition): Replica = {
+    getPartitionOrException(topicPartition, expectLeader = false).futureLocalReplicaOrException
   }
 
-  def getPartitionAndLeaderReplicaIfLocal(topicPartition: TopicPartition): (Partition, Replica) =  {
-    val partitionOpt = getPartition(topicPartition)
-    partitionOpt match {
-      case None if metadataCache.contains(topicPartition) =>
-        // The topic exists, but this broker is no longer a replica of it, so we return NOT_LEADER which
-        // forces clients to refresh metadata to find the new location. This can happen, for example,
-        // during a partition reassignment if a produce request from the client is sent to a broker after
-        // the local replica has been deleted.
-        throw new NotLeaderForPartitionException(s"Broker $localBrokerId is not a replica of $topicPartition")
-
-      case None =>
-        throw new UnknownTopicOrPartitionException(s"Partition $topicPartition doesn't exist")
-
-      case Some(partition) =>
-        if (partition eq ReplicaManager.OfflinePartition)
-          throw new KafkaStorageException(s"Partition $topicPartition is in an offline log directory on broker $localBrokerId")
-        else partition.leaderReplicaIfLocal match {
-          case Some(leaderReplica) => (partition, leaderReplica)
-          case None =>
-            throw new NotLeaderForPartitionException(s"Leader not local for partition $topicPartition on broker $localBrokerId")
-        }
-    }
+  def futureLocalReplica(topicPartition: TopicPartition): Option[Replica] = {
+    nonOfflinePartition(topicPartition).flatMap(_.futureLocalReplica)
   }
 
-  def getReplica(topicPartition: TopicPartition, replicaId: Int): Option[Replica] =
-    nonOfflinePartition(topicPartition).flatMap(_.getReplica(replicaId))
-
-  def getReplica(tp: TopicPartition): Option[Replica] = getReplica(tp, localBrokerId)
+  def localReplica(topicPartition: TopicPartition): Option[Replica] = {
+    nonOfflinePartition(topicPartition).flatMap(_.localReplica)
+  }
 
   def getLogDir(topicPartition: TopicPartition): Option[String] = {
-    getReplica(topicPartition).flatMap(_.log) match {
-      case Some(log) => Some(log.dir.getParent)
-      case None => None
-    }
+    localReplica(topicPartition).flatMap(_.log).map(_.dir.getParent)
   }
 
   /**
@@ -527,17 +518,9 @@ class ReplicaManager(val config: KafkaConfig,
         (topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(new InvalidTopicException(s"Cannot delete records of internal topic ${topicPartition.topic}"))))
       } else {
         try {
-          val (partition, replica) = getPartitionAndLeaderReplicaIfLocal(topicPartition)
-          val convertedOffset =
-            if (requestedOffset == DeleteRecordsRequest.HIGH_WATERMARK) {
-              replica.highWatermark.messageOffset
-            } else
-              requestedOffset
-          if (convertedOffset < 0)
-            throw new OffsetOutOfRangeException(s"The offset $convertedOffset for partition $topicPartition is not valid")
-
-          val lowWatermark = partition.deleteRecordsOnLeader(convertedOffset)
-          (topicPartition, LogDeleteRecordsResult(convertedOffset, lowWatermark))
+          val partition = getPartitionOrException(topicPartition, expectLeader = true)
+          val logDeleteResult = partition.deleteRecordsOnLeader(requestedOffset)
+          (topicPartition, logDeleteResult)
         } catch {
           case e@ (_: UnknownTopicOrPartitionException |
                    _: NotLeaderForPartitionException |
@@ -564,11 +547,10 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
-  /*
+  /**
    * For each pair of partition and log directory specified in the map, if the partition has already been created on
    * this broker, move its log files to the specified log directory. Otherwise, record the pair in the memory so that
    * the partition will be created in the specified log directory when broker receives LeaderAndIsrRequest for the partition later.
-   *
    */
   def alterReplicaLogDirs(partitionDirs: Map[TopicPartition, String]): Map[TopicPartition, Errors] = {
     replicaStateChangeLock synchronized {
@@ -577,18 +559,15 @@ class ReplicaManager(val config: KafkaConfig,
           if (!logManager.isLogDirOnline(destinationDir))
             throw new KafkaStorageException(s"Log directory $destinationDir is offline")
 
-          getPartition(topicPartition) match {
-            case Some(partition) =>
-              if (partition eq ReplicaManager.OfflinePartition)
-                throw new KafkaStorageException(s"Partition $topicPartition is offline")
+          getPartition(topicPartition).foreach { partition =>
+            if (partition eq ReplicaManager.OfflinePartition)
+              throw new KafkaStorageException(s"Partition $topicPartition is offline")
 
-              // Stop current replica movement if the destinationDir is different from the existing destination log directory
-              if (partition.futureReplicaDirChanged(destinationDir)) {
-                replicaAlterLogDirsManager.removeFetcherForPartitions(Set(topicPartition))
-                partition.removeFutureLocalReplica()
-              }
-
-            case None =>
+            // Stop current replica movement if the destinationDir is different from the existing destination log directory
+            if (partition.futureReplicaDirChanged(destinationDir)) {
+              replicaAlterLogDirsManager.removeFetcherForPartitions(Set(topicPartition))
+              partition.removeFutureLocalReplica()
+            }
           }
 
           // If the log for this partition has not been created yet:
@@ -596,19 +575,23 @@ class ReplicaManager(val config: KafkaConfig,
           //    when broker receives LeaderAndIsrRequest for this partition later.
           // 2) Respond with ReplicaNotAvailableException for this partition in the AlterReplicaLogDirsResponse
           logManager.maybeUpdatePreferredLogDir(topicPartition, destinationDir)
+
           // throw ReplicaNotAvailableException if replica does not exit for the given partition
-          getReplicaOrException(topicPartition)
+          val partition = getPartitionOrException(topicPartition, expectLeader = false)
+          partition.localReplicaOrException
 
           // If the destinationLDir is different from the current log directory of the replica:
           // - If there is no offline log directory, create the future log in the destinationDir (if it does not exist) and
           //   start ReplicaAlterDirThread to move data of this partition from the current log to the future log
           // - Otherwise, return KafkaStorageException. We do not create the future log while there is offline log directory
           //   so that we can avoid creating future log for the same partition in multiple log directories.
-          if (getPartition(topicPartition).get.maybeCreateFutureReplica(destinationDir)) {
-            val futureReplica = getReplicaOrException(topicPartition, Request.FutureLocalReplicaId)
+          if (partition.maybeCreateFutureReplica(destinationDir)) {
+            val futureReplica = futureLocalReplicaOrException(topicPartition)
             logManager.abortAndPauseCleaning(topicPartition)
-            replicaAlterLogDirsManager.addFetcherForPartitions(Map(topicPartition -> BrokerAndInitialOffset(
-              BrokerEndPoint(config.brokerId, "localhost", -1), futureReplica.highWatermark.messageOffset)))
+
+            val initialFetchState = InitialFetchState(BrokerEndPoint(config.brokerId, "localhost", -1),
+              partition.getLeaderEpoch, futureReplica.highWatermark.messageOffset)
+            replicaAlterLogDirsManager.addFetcherForPartitions(Map(topicPartition -> initialFetchState))
           }
 
           (topicPartition, Errors.NONE)
@@ -666,7 +649,7 @@ class ReplicaManager(val config: KafkaConfig,
   }
 
   def getLogEndOffsetLag(topicPartition: TopicPartition, logEndOffset: Long, isFuture: Boolean): Long = {
-    getReplica(topicPartition) match {
+    localReplica(topicPartition) match {
       case Some(replica) =>
         if (isFuture)
           replica.logEndOffset.messageOffset - logEndOffset
@@ -746,7 +729,7 @@ class ReplicaManager(val config: KafkaConfig,
           Some(new InvalidTopicException(s"Cannot append to internal topic ${topicPartition.topic}"))))
       } else {
         try {
-          val (partition, _) = getPartitionAndLeaderReplicaIfLocal(topicPartition)
+          val partition = getPartitionOrException(topicPartition, expectLeader = true)
           val info = partition.appendRecordsToLeader(records, isFromClient, requiredAcks)
           val numAppendedMessages = info.numMessages
 
@@ -756,7 +739,7 @@ class ReplicaManager(val config: KafkaConfig,
           brokerTopicStats.topicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
           brokerTopicStats.allTopicsStats.messagesInRate.mark(numAppendedMessages)
 
-          trace(s"${records.sizeInBytes} written to log ${topicPartition} beginning at offset " +
+          trace(s"${records.sizeInBytes} written to log $topicPartition beginning at offset " +
             s"${info.firstOffset.getOrElse(-1)} and ending at offset ${info.lastOffset}")
           (topicPartition, LogAppendResult(info))
         } catch {
@@ -779,13 +762,31 @@ class ReplicaManager(val config: KafkaConfig,
             }
             brokerTopicStats.topicStats(topicPartition.topic).failedProduceRequestRate.mark()
             brokerTopicStats.allTopicsStats.failedProduceRequestRate.mark()
-            error("Error processing append operation on partition %s".format(topicPartition), t)
+            error(s"Error processing append operation on partition $topicPartition", t)
             (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t)))
         }
       }
     }
   }
 
+  def fetchOffsetForTimestamp(topicPartition: TopicPartition,
+                              timestamp: Long,
+                              isolationLevel: Option[IsolationLevel],
+                              currentLeaderEpoch: Optional[Integer],
+                              fetchOnlyFromLeader: Boolean): TimestampOffset = {
+    val partition = getPartitionOrException(topicPartition, expectLeader = fetchOnlyFromLeader)
+    partition.fetchOffsetForTimestamp(timestamp, isolationLevel, currentLeaderEpoch, fetchOnlyFromLeader)
+  }
+
+  def legacyFetchOffsetsForTimestamp(topicPartition: TopicPartition,
+                                     timestamp: Long,
+                                     maxNumOffsets: Int,
+                                     isFromConsumer: Boolean,
+                                     fetchOnlyFromLeader: Boolean): Seq[Long] = {
+    val partition = getPartitionOrException(topicPartition, expectLeader = fetchOnlyFromLeader)
+    partition.legacyFetchOffsetsForTimestamp(timestamp, maxNumOffsets, isFromConsumer, fetchOnlyFromLeader)
+  }
+
   /**
    * Fetch messages from the leader replica, and wait until enough data can be fetched and return;
    * the callback function will be triggered either when timeout or required fetch info is satisfied
@@ -801,18 +802,24 @@ class ReplicaManager(val config: KafkaConfig,
                     isolationLevel: IsolationLevel) {
     val isFromFollower = Request.isValidBrokerId(replicaId)
     val fetchOnlyFromLeader = replicaId != Request.DebuggingConsumerId && replicaId != Request.FutureLocalReplicaId
-    val fetchOnlyCommitted = !isFromFollower && replicaId != Request.FutureLocalReplicaId
+
+    val fetchIsolation = if (isFromFollower || replicaId == Request.FutureLocalReplicaId)
+      FetchLogEnd
+    else if (isolationLevel == IsolationLevel.READ_COMMITTED)
+      FetchTxnCommitted
+    else
+      FetchHighWatermark
+
 
     def readFromLog(): Seq[(TopicPartition, LogReadResult)] = {
       val result = readFromLocalLog(
         replicaId = replicaId,
         fetchOnlyFromLeader = fetchOnlyFromLeader,
-        readOnlyCommitted = fetchOnlyCommitted,
+        fetchIsolation = fetchIsolation,
         fetchMaxBytes = fetchMaxBytes,
         hardMaxBytesLimit = hardMaxBytesLimit,
         readPartitionInfo = fetchInfos,
-        quota = quota,
-        isolationLevel = isolationLevel)
+        quota = quota)
       if (isFromFollower) updateFollowerLogReadResults(replicaId, result)
       else result
     }
@@ -844,8 +851,8 @@ class ReplicaManager(val config: KafkaConfig,
         (topicPartition, FetchPartitionStatus(result.info.fetchOffsetMetadata, fetchInfo))
       }
       val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit, fetchOnlyFromLeader,
-        fetchOnlyCommitted, isFromFollower, replicaId, fetchPartitionStatus)
-      val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, isolationLevel, responseCallback)
+        fetchIsolation, isFromFollower, replicaId, fetchPartitionStatus)
+      val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, responseCallback)
 
       // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
       val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => new TopicPartitionOperationKey(tp) }
@@ -862,12 +869,11 @@ class ReplicaManager(val config: KafkaConfig,
    */
   def readFromLocalLog(replicaId: Int,
                        fetchOnlyFromLeader: Boolean,
-                       readOnlyCommitted: Boolean,
+                       fetchIsolation: FetchIsolation,
                        fetchMaxBytes: Int,
                        hardMaxBytesLimit: Boolean,
                        readPartitionInfo: Seq[(TopicPartition, PartitionData)],
-                       quota: ReplicaQuota,
-                       isolationLevel: IsolationLevel): Seq[(TopicPartition, LogReadResult)] = {
+                       quota: ReplicaQuota): Seq[(TopicPartition, LogReadResult)] = {
 
     def read(tp: TopicPartition, fetchInfo: PartitionData, limitBytes: Int, minOneMessage: Boolean): LogReadResult = {
       val offset = fetchInfo.fetchOffset
@@ -882,68 +888,46 @@ class ReplicaManager(val config: KafkaConfig,
           s"remaining response limit $limitBytes" +
           (if (minOneMessage) s", ignoring response/partition size limits" else ""))
 
-        // decide whether to only fetch from leader
-        val localReplica = if (fetchOnlyFromLeader)
-          getLeaderReplicaIfLocal(tp)
-        else
-          getReplicaOrException(tp)
-
-        val initialHighWatermark = localReplica.highWatermark.messageOffset
-        val lastStableOffset = if (isolationLevel == IsolationLevel.READ_COMMITTED)
-          Some(localReplica.lastStableOffset.messageOffset)
-        else
-          None
-
-        // decide whether to only fetch committed data (i.e. messages below high watermark)
-        val maxOffsetOpt = if (readOnlyCommitted)
-          Some(lastStableOffset.getOrElse(initialHighWatermark))
-        else
-          None
-
-        /* Read the LogOffsetMetadata prior to performing the read from the log.
-         * We use the LogOffsetMetadata to determine if a particular replica is in-sync or not.
-         * Using the log end offset after performing the read can lead to a race condition
-         * where data gets appended to the log immediately after the replica has consumed from it
-         * This can cause a replica to always be out of sync.
-         */
-        val initialLogEndOffset = localReplica.logEndOffset.messageOffset
-        val initialLogStartOffset = localReplica.logStartOffset
+        val partition = getPartitionOrException(tp, expectLeader = fetchOnlyFromLeader)
+        val adjustedMaxBytes = math.min(fetchInfo.maxBytes, limitBytes)
         val fetchTimeMs = time.milliseconds
-        val logReadInfo = localReplica.log match {
-          case Some(log) =>
-            val adjustedFetchSize = math.min(partitionFetchSize, limitBytes)
-
-            // Try the read first, this tells us whether we need all of adjustedFetchSize for this partition
-            val fetch = log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage, isolationLevel)
-
-            // If the partition is being throttled, simply return an empty set.
-            if (shouldLeaderThrottle(quota, tp, replicaId))
-              FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
-            // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
-            // progress in such cases and don't need to report a `RecordTooLargeException`
-            else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete)
-              FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
-            else fetch
 
-          case None =>
-            error(s"Leader for partition $tp does not have a local log")
-            FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)
+        // Try the read first, this tells us whether we need all of adjustedFetchSize for this partition
+        val readInfo = partition.readRecords(
+          fetchOffset = fetchInfo.fetchOffset,
+          currentLeaderEpoch = fetchInfo.currentLeaderEpoch,
+          maxBytes = adjustedMaxBytes,
+          fetchIsolation = fetchIsolation,
+          fetchOnlyFromLeader = fetchOnlyFromLeader,
+          minOneMessage = minOneMessage)
+
+        val fetchDataInfo = if (shouldLeaderThrottle(quota, tp, replicaId)) {
+          // If the partition is being throttled, simply return an empty set.
+          FetchDataInfo(readInfo.fetchedData.fetchOffsetMetadata, MemoryRecords.EMPTY)
+        } else if (!hardMaxBytesLimit && readInfo.fetchedData.firstEntryIncomplete) {
+          // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
+          // progress in such cases and don't need to report a `RecordTooLargeException`
+          FetchDataInfo(readInfo.fetchedData.fetchOffsetMetadata, MemoryRecords.EMPTY)
+        } else {
+          readInfo.fetchedData
         }
 
-        LogReadResult(info = logReadInfo,
-                      highWatermark = initialHighWatermark,
-                      leaderLogStartOffset = initialLogStartOffset,
-                      leaderLogEndOffset = initialLogEndOffset,
+        LogReadResult(info = fetchDataInfo,
+                      highWatermark = readInfo.highWatermark,
+                      leaderLogStartOffset = readInfo.logStartOffset,
+                      leaderLogEndOffset = readInfo.logEndOffset,
                       followerLogStartOffset = followerLogStartOffset,
                       fetchTimeMs = fetchTimeMs,
-                      readSize = partitionFetchSize,
-                      lastStableOffset = lastStableOffset,
+                      readSize = adjustedMaxBytes,
+                      lastStableOffset = Some(readInfo.lastStableOffset),
                       exception = None)
       } catch {
         // NOTE: Failed fetch requests metric is not incremented for known exceptions since it
         // is supposed to indicate un-expected failure of a broker in handling a fetch request
         case e@ (_: UnknownTopicOrPartitionException |
                  _: NotLeaderForPartitionException |
+                 _: UnknownLeaderEpochException |
+                 _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
                  _: OffsetOutOfRangeException) =>
@@ -953,7 +937,7 @@ class ReplicaManager(val config: KafkaConfig,
                         leaderLogEndOffset = -1L,
                         followerLogStartOffset = -1L,
                         fetchTimeMs = -1L,
-                        readSize = partitionFetchSize,
+                        readSize = 0,
                         lastStableOffset = None,
                         exception = Some(e))
         case e: Throwable =>
@@ -966,7 +950,7 @@ class ReplicaManager(val config: KafkaConfig,
                         leaderLogEndOffset = -1L,
                         followerLogStartOffset = -1L,
                         fetchTimeMs = -1L,
-                        readSize = partitionFetchSize,
+                        readSize = 0,
                         lastStableOffset = None,
                         exception = Some(e))
       }
@@ -998,7 +982,7 @@ class ReplicaManager(val config: KafkaConfig,
     quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
   }
 
-  def getLogConfig(topicPartition: TopicPartition): Option[LogConfig] = getReplica(topicPartition).flatMap(_.log.map(_.config))
+  def getLogConfig(topicPartition: TopicPartition): Option[LogConfig] = localReplica(topicPartition).flatMap(_.log.map(_.config))
 
   def getMagic(topicPartition: TopicPartition): Option[Byte] = getLogConfig(topicPartition).map(_.messageFormatVersion.recordVersion.value)
 
@@ -1039,18 +1023,23 @@ class ReplicaManager(val config: KafkaConfig,
 
         // First check partition's leader epoch
         val partitionState = new mutable.HashMap[Partition, LeaderAndIsrRequest.PartitionState]()
-        val newPartitions = leaderAndIsrRequest.partitionStates.asScala.keys.filter(topicPartition => getPartition(topicPartition).isEmpty)
+        val newPartitions = new mutable.HashSet[Partition]
 
         leaderAndIsrRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) =>
-          val partition = getOrCreatePartition(topicPartition)
-          val partitionLeaderEpoch = partition.getLeaderEpoch
+          val partition = getPartition(topicPartition).getOrElse {
+            val createdPartition = getOrCreatePartition(topicPartition)
+            newPartitions.add(createdPartition)
+            createdPartition
+          }
+          val currentLeaderEpoch = partition.getLeaderEpoch
+          val requestLeaderEpoch = stateInfo.basePartitionState.leaderEpoch
           if (partition eq ReplicaManager.OfflinePartition) {
             stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from " +
               s"controller $controllerId with correlation id $correlationId " +
               s"epoch $controllerEpoch for partition $topicPartition as the local replica for the " +
               "partition is in an offline log directory")
             responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR)
-          } else if (partitionLeaderEpoch < stateInfo.basePartitionState.leaderEpoch) {
+          } else if (requestLeaderEpoch > currentLeaderEpoch) {
             // If the leader epoch is valid record the epoch of the controller that made the leadership decision.
             // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path
             if(stateInfo.basePartitionState.replicas.contains(localBrokerId))
@@ -1066,8 +1055,8 @@ class ReplicaManager(val config: KafkaConfig,
             stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from " +
               s"controller $controllerId with correlation id $correlationId " +
               s"epoch $controllerEpoch for partition $topicPartition since its associated " +
-              s"leader epoch ${stateInfo.basePartitionState.leaderEpoch} is not higher than the current " +
-              s"leader epoch $partitionLeaderEpoch")
+              s"leader epoch $requestLeaderEpoch is not higher than the current " +
+              s"leader epoch $currentLeaderEpoch")
             responseMap.put(topicPartition, Errors.STALE_CONTROLLER_EPOCH)
           }
         }
@@ -1086,16 +1075,16 @@ class ReplicaManager(val config: KafkaConfig,
         else
           Set.empty[Partition]
 
-        leaderAndIsrRequest.partitionStates.asScala.keys.foreach(topicPartition =>
+        leaderAndIsrRequest.partitionStates.asScala.keys.foreach { topicPartition =>
           /*
            * If there is offline log directory, a Partition object may have been created by getOrCreatePartition()
            * before getOrCreateReplica() failed to create local replica due to KafkaStorageException.
            * In this case ReplicaManager.allPartitions will map this topic-partition to an empty Partition object.
            * we need to map this topic-partition to OfflinePartition instead.
            */
-          if (getReplica(topicPartition).isEmpty && (allPartitions.get(topicPartition) ne ReplicaManager.OfflinePartition))
+          if (localReplica(topicPartition).isEmpty && (allPartitions.get(topicPartition) ne ReplicaManager.OfflinePartition))
             allPartitions.put(topicPartition, ReplicaManager.OfflinePartition)
-        )
+        }
 
         // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions
         // have been completely populated before starting the checkpointing there by avoiding weird race conditions
@@ -1104,17 +1093,25 @@ class ReplicaManager(val config: KafkaConfig,
           hwThreadInitialized = true
         }
 
-        val newOnlineReplicas = newPartitions.flatMap(topicPartition => getReplica(topicPartition))
-        // Add future replica to partition's map
-        val futureReplicasAndInitialOffset = newOnlineReplicas.filter { replica =>
-          logManager.getLog(replica.topicPartition, isFuture = true).isDefined
-        }.map { replica =>
-          replica.topicPartition -> BrokerAndInitialOffset(BrokerEndPoint(config.brokerId, "localhost", -1), replica.highWatermark.messageOffset)
-        }.toMap
-        futureReplicasAndInitialOffset.keys.foreach(tp => getPartition(tp).get.getOrCreateReplica(Request.FutureLocalReplicaId))
+        val futureReplicasAndInitialOffset = new mutable.HashMap[TopicPartition, InitialFetchState]
+        for (partition <- newPartitions) {
+          val topicPartition = partition.topicPartition
+          if (logManager.getLog(topicPartition, isFuture = true).isDefined) {
+            partition.localReplica.foreach { replica =>
+              val leader = BrokerEndPoint(config.brokerId, "localhost", -1)
+
+              // Add future replica to partition's map
+              partition.getOrCreateReplica(Request.FutureLocalReplicaId, isNew = false)
+
+              // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move
+              // replica from source dir to destination dir
+              logManager.abortAndPauseCleaning(topicPartition)
 
-        // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move replica from source dir to destination dir
-        futureReplicasAndInitialOffset.keys.foreach(logManager.abortAndPauseCleaning)
+              futureReplicasAndInitialOffset.put(topicPartition, InitialFetchState(leader,
+                partition.getLeaderEpoch, replica.highWatermark.messageOffset))
+            }
+          }
+        }
         replicaAlterLogDirsManager.addFetcherForPartitions(futureReplicasAndInitialOffset)
 
         replicaFetcherManager.shutdownIdleFetcherThreads()
@@ -1235,7 +1232,6 @@ class ReplicaManager(val config: KafkaConfig,
     val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set()
 
     try {
-
       // TODO: Delete leaders from LeaderAndIsrRequest
       partitionStates.foreach { case (partition, partitionStateInfo) =>
         val newLeaderBrokerId = partitionStateInfo.basePartitionState.leader
@@ -1260,7 +1256,7 @@ class ReplicaManager(val config: KafkaConfig,
                 s"but cannot become follower since the new leader $newLeaderBrokerId is unavailable.")
               // Create the local replica even if the leader is unavailable. This is required to ensure that we include
               // the partition's high watermark in the checkpoint file (see KAFKA-1647)
-              partition.getOrCreateReplica(isNew = partitionStateInfo.isNew)
+              partition.getOrCreateReplica(localBrokerId, isNew = partitionStateInfo.isNew)
           }
         } catch {
           case e: KafkaStorageException =>
@@ -1304,10 +1300,12 @@ class ReplicaManager(val config: KafkaConfig,
       }
       else {
         // we do not need to check if the leader exists again since this has been done at the beginning of this process
-        val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition =>
-          partition.topicPartition -> BrokerAndInitialOffset(
-            metadataCache.getAliveBrokers.find(_.id == partition.leaderReplicaIdOpt.get).get.brokerEndPoint(config.interBrokerListenerName),
-            partition.getReplica().get.highWatermark.messageOffset)).toMap
+        val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map { partition =>
+          val leader = metadataCache.getAliveBrokers.find(_.id == partition.leaderReplicaIdOpt.get).get
+            .brokerEndPoint(config.interBrokerListenerName)
+          val fetchOffset = partition.localReplicaOrException.highWatermark.messageOffset
+          partition.topicPartition -> InitialFetchState(leader, partition.getLeaderEpoch, fetchOffset)
+        }.toMap
         replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset)
 
         partitionsToMakeFollower.foreach { partition =>
@@ -1381,8 +1379,8 @@ class ReplicaManager(val config: KafkaConfig,
   def checkpointHighWatermarks() {
     val replicas = nonOfflinePartitionsIterator.flatMap { partition =>
       val replicasList: mutable.Set[Replica] = mutable.Set()
-      partition.getReplica(localBrokerId).foreach(replicasList.add)
-      partition.getReplica(Request.FutureLocalReplicaId).foreach(replicasList.add)
+      partition.localReplica.foreach(replicasList.add)
+      partition.futureLocalReplica.foreach(replicasList.add)
       replicasList
     }.filter(_.log.isDefined).toBuffer
     val replicasByDir = replicas.groupBy(_.log.get.dir.getParent)
@@ -1409,13 +1407,13 @@ class ReplicaManager(val config: KafkaConfig,
     info(s"Stopping serving replicas in dir $dir")
     replicaStateChangeLock synchronized {
       val newOfflinePartitions = nonOfflinePartitionsIterator.filter { partition =>
-        partition.getReplica(config.brokerId).exists { replica =>
+        partition.localReplica.exists { replica =>
           replica.log.isDefined && replica.log.get.dir.getParent == dir
         }
       }.map(_.topicPartition).toSet
 
       val partitionsWithOfflineFutureReplica = nonOfflinePartitionsIterator.filter { partition =>
-        partition.getReplica(Request.FutureLocalReplicaId).exists { replica =>
+        partition.futureLocalReplica.exists { replica =>
           replica.log.isDefined && replica.log.get.dir.getParent == dir
         }
       }.toSet
@@ -1482,7 +1480,8 @@ class ReplicaManager(val config: KafkaConfig,
           if (partition eq ReplicaManager.OfflinePartition)
             new EpochEndOffset(Errors.KAFKA_STORAGE_ERROR, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
           else
-            partition.lastOffsetForLeaderEpoch(partitionData.leaderEpoch)
+            partition.lastOffsetForLeaderEpoch(partitionData.currentLeaderEpoch, partitionData.leaderEpoch,
+              fetchOnlyFromLeader = true)
 
         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/utils/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala
index 9d92b97..cfb8771 100644
--- a/core/src/main/scala/kafka/utils/DelayedItem.scala
+++ b/core/src/main/scala/kafka/utils/DelayedItem.scala
@@ -23,7 +23,7 @@ import org.apache.kafka.common.utils.Time
 
 import scala.math._
 
-class DelayedItem(delayMs: Long) extends Delayed with Logging {
+class DelayedItem(val delayMs: Long) extends Delayed with Logging {
 
   private val dueMs = Time.SYSTEM.milliseconds + delayMs
 
diff --git a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala
index 672a669..ca23e1f 100644
--- a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala
@@ -804,7 +804,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
     assertEquals(3L, lowWatermark)
 
     for (i <- 0 until serverCount)
-      assertEquals(3, servers(i).replicaManager.getReplica(topicPartition).get.logStartOffset)
+      assertEquals(3, servers(i).replicaManager.localReplica(topicPartition).get.logStartOffset)
   }
 
   @Test
@@ -813,16 +813,16 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
     val followerIndex = if (leaders(0) != servers(0).config.brokerId) 0 else 1
 
     def waitForFollowerLog(expectedStartOffset: Long, expectedEndOffset: Long): Unit = {
-      TestUtils.waitUntilTrue(() => servers(followerIndex).replicaManager.getReplica(topicPartition) != None,
+      TestUtils.waitUntilTrue(() => servers(followerIndex).replicaManager.localReplica(topicPartition) != None,
                               "Expected follower to create replica for partition")
 
       // wait until the follower discovers that log start offset moved beyond its HW
       TestUtils.waitUntilTrue(() => {
-        servers(followerIndex).replicaManager.getReplica(topicPartition).get.logStartOffset == expectedStartOffset
+        servers(followerIndex).replicaManager.localReplica(topicPartition).get.logStartOffset == expectedStartOffset
       }, s"Expected follower to discover new log start offset $expectedStartOffset")
 
       TestUtils.waitUntilTrue(() => {
-        servers(followerIndex).replicaManager.getReplica(topicPartition).get.logEndOffset.messageOffset == expectedEndOffset
+        servers(followerIndex).replicaManager.localReplica(topicPartition).get.logEndOffset.messageOffset == expectedEndOffset
       }, s"Expected follower to catch up to log end offset $expectedEndOffset")
     }
 
@@ -843,7 +843,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
 
     // after the new replica caught up, all replicas should have same log start offset
     for (i <- 0 until serverCount)
-      assertEquals(3, servers(i).replicaManager.getReplica(topicPartition).get.logStartOffset)
+      assertEquals(3, servers(i).replicaManager.localReplica(topicPartition).get.logStartOffset)
 
     // kill the same follower again, produce more records, and delete records beyond follower's LOE
     killBroker(followerIndex)
@@ -867,8 +867,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
     result.all().get()
     // make sure we are in the expected state after delete records
     for (i <- 0 until serverCount) {
-      assertEquals(3, servers(i).replicaManager.getReplica(topicPartition).get.logStartOffset)
-      assertEquals(expectedLEO, servers(i).replicaManager.getReplica(topicPartition).get.logEndOffset.messageOffset)
+      assertEquals(3, servers(i).replicaManager.localReplica(topicPartition).get.logStartOffset)
+      assertEquals(expectedLEO, servers(i).replicaManager.localReplica(topicPartition).get.logEndOffset.messageOffset)
     }
 
     // we will create another dir just for one server
@@ -882,8 +882,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
     }, "timed out waiting for replica movement")
 
     // once replica moved, its LSO and LEO should match other replicas
-    assertEquals(3, servers(0).replicaManager.getReplica(topicPartition).get.logStartOffset)
-    assertEquals(expectedLEO, servers(0).replicaManager.getReplica(topicPartition).get.logEndOffset.messageOffset)
+    assertEquals(3, servers(0).replicaManager.localReplica(topicPartition).get.logStartOffset)
+    assertEquals(expectedLEO, servers(0).replicaManager.localReplica(topicPartition).get.logEndOffset.messageOffset)
   }
 
   @Test
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index dbe2a02..ae62736 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -278,8 +278,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   }
 
   private def offsetsForLeaderEpochRequest: OffsetsForLeaderEpochRequest = {
-    new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion)
-      .add(tp, Optional.of(27), 7).build()
+    val epochs = Map(tp -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.of(27), 7))
+    new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, epochs.asJava).build()
   }
 
   private def createOffsetFetchRequest = {
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
index dabe5a4..46bd81c 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
@@ -129,7 +129,7 @@ class ConsumerBounceTest extends BaseRequestTest with Logging {
 
     // wait until all the followers have synced the last HW with leader
     TestUtils.waitUntilTrue(() => servers.forall(server =>
-      server.replicaManager.getReplica(tp).get.highWatermark.messageOffset == numRecords
+      server.replicaManager.localReplica(tp).get.highWatermark.messageOffset == numRecords
     ), "Failed to update high watermark for followers after timeout")
 
     val scheduler = new BounceBrokerScheduler(numIters)
diff --git a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala
new file mode 100644
index 0000000..890ea3b
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import java.util.Optional
+
+import kafka.cluster.Partition
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.FencedLeaderEpochException
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.requests.{FetchRequest, IsolationLevel}
+import org.easymock.{EasyMock, EasyMockSupport}
+import org.junit.Test
+import org.junit.Assert._
+
+class DelayedFetchTest extends EasyMockSupport {
+  private val maxBytes = 1024
+  private val replicaManager = mock(classOf[ReplicaManager])
+  private val replicaQuota = mock(classOf[ReplicaQuota])
+
+  @Test
+  def testFetchWithFencedEpoch(): Unit = {
+    val topicPartition = new TopicPartition("topic", 0)
+    val fetchOffset = 500L
+    val logStartOffset = 0L
+    val currentLeaderEpoch = Optional.of[Integer](10)
+    val replicaId = 1
+
+    val fetchStatus = FetchPartitionStatus(
+      startOffsetMetadata = LogOffsetMetadata(fetchOffset),
+      fetchInfo = new FetchRequest.PartitionData(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch))
+    val fetchMetadata = buildFetchMetadata(replicaId, topicPartition, fetchStatus)
+
+    var fetchResultOpt: Option[FetchPartitionData] = None
+    def callback(responses: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
+      fetchResultOpt = Some(responses.head._2)
+    }
+
+    val delayedFetch = new DelayedFetch(
+      delayMs = 500,
+      fetchMetadata = fetchMetadata,
+      replicaManager = replicaManager,
+      quota = replicaQuota,
+      responseCallback = callback)
+
+    val partition = mock(classOf[Partition])
+
+    EasyMock.expect(replicaManager.getPartitionOrException(topicPartition, expectLeader = true))
+        .andReturn(partition)
+    EasyMock.expect(partition.fetchOffsetSnapshot(currentLeaderEpoch, fetchOnlyFromLeader = true))
+        .andThrow(new FencedLeaderEpochException("Requested epoch has been fenced"))
+
+    expectReadFromReplicaWithError(replicaId, topicPartition, fetchStatus.fetchInfo, Errors.FENCED_LEADER_EPOCH)
+
+    replayAll()
+
+    assertTrue(delayedFetch.tryComplete())
+    assertTrue(delayedFetch.isCompleted)
+    assertTrue(fetchResultOpt.isDefined)
+
+    val fetchResult = fetchResultOpt.get
+    assertEquals(Errors.FENCED_LEADER_EPOCH, fetchResult.error)
+  }
+
+  private def buildFetchMetadata(replicaId: Int,
+                                 topicPartition: TopicPartition,
+                                 fetchStatus: FetchPartitionStatus): FetchMetadata = {
+    FetchMetadata(fetchMinBytes = 1,
+      fetchMaxBytes = maxBytes,
+      hardMaxBytesLimit = false,
+      fetchOnlyLeader = true,
+      fetchIsolation = FetchLogEnd,
+      isFromFollower = true,
+      replicaId = replicaId,
+      fetchPartitionStatus = Seq((topicPartition, fetchStatus)))
+  }
+
+  private def expectReadFromReplicaWithError(replicaId: Int,
+                                             topicPartition: TopicPartition,
+                                             fetchPartitionData: FetchRequest.PartitionData,
+                                             error: Errors): Unit = {
+    EasyMock.expect(replicaManager.readFromLocalLog(
+      replicaId = replicaId,
+      fetchOnlyFromLeader = true,
+      fetchIsolation = FetchLogEnd,
+      fetchMaxBytes = maxBytes,
+      hardMaxBytesLimit = false,
+      readPartitionInfo = Seq((topicPartition, fetchPartitionData)),
+      quota = replicaQuota))
+      .andReturn(Seq((topicPartition, buildReadResultWithError(error))))
+  }
+
+  private def buildReadResultWithError(error: Errors): LogReadResult = {
+    LogReadResult(
+      exception = Some(error.exception),
+      info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY),
+      highWatermark = -1L,
+      leaderLogStartOffset = -1L,
+      leaderLogEndOffset = -1L,
+      followerLogStartOffset = -1L,
+      fetchTimeMs = -1L,
+      readSize = -1,
+      lastStableOffset = None)
+  }
+
+}
diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
index ff4ccef..787cc8d 100644
--- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
@@ -622,10 +622,10 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
         val replicaFetcherManager = servers(i).replicaManager.replicaFetcherManager
         val truncationOffset = tp.partition
         replicaFetcherManager.markPartitionsForTruncation(leaderId, tp, truncationOffset)
-        val fetcherThreads = replicaFetcherManager.fetcherThreadMap.filter(_._2.partitionStates.contains(tp))
+        val fetcherThreads = replicaFetcherManager.fetcherThreadMap.filter(_._2.fetchState(tp).isDefined)
         assertEquals(1, fetcherThreads.size)
-        assertEquals(replicaFetcherManager.getFetcherId(tp.topic, tp.partition), fetcherThreads.head._1.fetcherId)
-        assertEquals(truncationOffset, fetcherThreads.head._2.partitionStates.stateValue(tp).fetchOffset)
+        assertEquals(replicaFetcherManager.getFetcherId(tp), fetcherThreads.head._1.fetcherId)
+        assertEquals(Some(truncationOffset), fetcherThreads.head._2.fetchState(tp).map(_.fetchOffset))
       }
     }
   }
diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala
index 54f8582..87e26fe 100755
--- a/core/src/test/scala/other/kafka/StressTestLog.scala
+++ b/core/src/test/scala/other/kafka/StressTestLog.scala
@@ -132,7 +132,11 @@ object StressTestLog {
   class ReaderThread(val log: Log) extends WorkerThread with LogProgress {
     override def work() {
       try {
-        log.read(currentOffset, 1024, Some(currentOffset + 1), isolationLevel = IsolationLevel.READ_UNCOMMITTED).records match {
+        log.read(currentOffset,
+          maxLength = 1024,
+          maxOffset = Some(currentOffset + 1),
+          minOneMessage = true,
+          includeAbortedTxns = false).records match {
           case read: FileRecords if read.sizeInBytes > 0 => {
             val first = read.batches.iterator.next()
             require(first.lastOffset == currentOffset, "We should either read nothing or the message we asked for.")
diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
index b44c239..1438da1 100644
--- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
@@ -99,13 +99,14 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
       "broker 101 should be the new leader", pause = 1L
     )
 
-    assertEquals(100, newLeaderServer.replicaManager.getReplicaOrException(topicPartition).highWatermark.messageOffset)
+    assertEquals(100, newLeaderServer.replicaManager.localReplicaOrException(topicPartition)
+      .highWatermark.messageOffset)
     val newFollowerServer = servers.find(_.config.brokerId == 102).get
-    TestUtils.waitUntilTrue(() => newFollowerServer.replicaManager.getReplicaOrException(topicPartition).highWatermark.messageOffset == 100,
+    TestUtils.waitUntilTrue(() => newFollowerServer.replicaManager.localReplicaOrException(topicPartition)
+      .highWatermark.messageOffset == 100,
       "partition follower's highWatermark should be 100")
   }
 
-
   @Test
   def shouldMoveSinglePartition(): Unit = {
     //Given a single replica on server 100
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index 7cdc778..b5b271e 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -18,22 +18,23 @@ package kafka.cluster
 
 import java.io.File
 import java.nio.ByteBuffer
-import java.util.Properties
+import java.util.{Optional, Properties}
 import java.util.concurrent.CountDownLatch
 import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.api.Request
 import kafka.common.UnexpectedAppendOffsetException
-import kafka.log.{LogConfig, LogManager, CleanerConfig}
+import kafka.log.{Defaults => _, _}
 import kafka.server._
-import kafka.utils.{MockTime, TestUtils, MockScheduler}
+import kafka.utils.{MockScheduler, MockTime, TestUtils}
 import kafka.zk.KafkaZkClient
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.ReplicaNotAvailableException
 import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.common.record._
-import org.apache.kafka.common.requests.LeaderAndIsrRequest
+import org.apache.kafka.common.requests.{IsolationLevel, LeaderAndIsrRequest, ListOffsetRequest}
 import org.junit.{After, Before, Test}
 import org.junit.Assert._
 import org.scalatest.Assertions.assertThrows
@@ -99,12 +100,6 @@ class PartitionTest {
 
   @Test
   def testMakeLeaderUpdatesEpochCache(): Unit = {
-    val controllerEpoch = 3
-    val leader = brokerId
-    val follower = brokerId + 1
-    val controllerId = brokerId + 3
-    val replicas = List[Integer](leader, follower).asJava
-    val isr = List[Integer](leader, follower).asJava
     val leaderEpoch = 8
 
     val log = logManager.getOrCreateLog(topicPartition, logConfig)
@@ -118,14 +113,11 @@ class PartitionTest {
     ), leaderEpoch = 5)
     assertEquals(4, log.logEndOffset)
 
-    val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
-    assertTrue("Expected makeLeader to succeed",
-      partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch,
-        isr, 1, replicas, true), 0))
-
+    val partition = setupPartitionWithMocks(leaderEpoch = leaderEpoch, isLeader = true, log = log)
     assertEquals(Some(4), partition.leaderReplicaIfLocal.map(_.logEndOffset.messageOffset))
 
-    val epochEndOffset = partition.lastOffsetForLeaderEpoch(leaderEpoch)
+    val epochEndOffset = partition.lastOffsetForLeaderEpoch(currentLeaderEpoch = Optional.of[Integer](leaderEpoch),
+      leaderEpoch = leaderEpoch, fetchOnlyFromLeader = true)
     assertEquals(4, epochEndOffset.endOffset)
     assertEquals(leaderEpoch, epochEndOffset.leaderEpoch)
   }
@@ -141,12 +133,12 @@ class PartitionTest {
     val log2 = logManager.getOrCreateLog(topicPartition, logConfig, isFuture = true)
     val currentReplica = new Replica(brokerId, topicPartition, time, log = Some(log1))
     val futureReplica = new Replica(Request.FutureLocalReplicaId, topicPartition, time, log = Some(log2))
-    val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
+    val partition = Partition(topicPartition, time, replicaManager)
 
     partition.addReplicaIfNotExists(futureReplica)
     partition.addReplicaIfNotExists(currentReplica)
-    assertEquals(Some(currentReplica), partition.getReplica(brokerId))
-    assertEquals(Some(futureReplica), partition.getReplica(Request.FutureLocalReplicaId))
+    assertEquals(Some(currentReplica), partition.localReplica)
+    assertEquals(Some(futureReplica), partition.futureLocalReplica)
 
     val thread1 = new Thread {
       override def run(): Unit = {
@@ -168,17 +160,261 @@ class PartitionTest {
     latch.countDown()
     thread1.join()
     thread2.join()
-    assertEquals(None, partition.getReplica(Request.FutureLocalReplicaId))
+    assertEquals(None, partition.futureLocalReplica)
+  }
+
+  @Test
+  def testFetchOffsetSnapshotEpochValidationForLeader(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
+
+    def assertSnapshotError(expectedError: Errors, currentLeaderEpoch: Optional[Integer]): Unit = {
+      partition.fetchOffsetSnapshotOrError(currentLeaderEpoch, fetchOnlyFromLeader = true) match {
+        case Left(_) => assertEquals(Errors.NONE, expectedError)
+        case Right(error) => assertEquals(expectedError, error)
+      }
+    }
+
+    assertSnapshotError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
+    assertSnapshotError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
+    assertSnapshotError(Errors.NONE, Optional.of(leaderEpoch))
+    assertSnapshotError(Errors.NONE, Optional.empty())
+  }
+
+  @Test
+  def testFetchOffsetSnapshotEpochValidationForFollower(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
+
+    def assertSnapshotError(expectedError: Errors,
+                            currentLeaderEpoch: Optional[Integer],
+                            fetchOnlyLeader: Boolean): Unit = {
+      partition.fetchOffsetSnapshotOrError(currentLeaderEpoch, fetchOnlyFromLeader = fetchOnlyLeader) match {
+        case Left(_) => assertEquals(expectedError, Errors.NONE)
+        case Right(error) => assertEquals(expectedError, error)
+      }
+    }
+
+    assertSnapshotError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
+    assertSnapshotError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
+    assertSnapshotError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
+    assertSnapshotError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
+
+    assertSnapshotError(Errors.NOT_LEADER_FOR_PARTITION, Optional.of(leaderEpoch), fetchOnlyLeader = true)
+    assertSnapshotError(Errors.NOT_LEADER_FOR_PARTITION, Optional.empty(), fetchOnlyLeader = true)
+    assertSnapshotError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
+    assertSnapshotError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
+  }
+
+  @Test
+  def testOffsetForLeaderEpochValidationForLeader(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
+
+    def assertLastOffsetForLeaderError(error: Errors, currentLeaderEpochOpt: Optional[Integer]): Unit = {
+      val endOffset = partition.lastOffsetForLeaderEpoch(currentLeaderEpochOpt, 0,
+        fetchOnlyFromLeader = true)
+      assertEquals(error, endOffset.error)
+    }
+
+    assertLastOffsetForLeaderError(Errors.NONE, Optional.empty())
+    assertLastOffsetForLeaderError(Errors.NONE, Optional.of(leaderEpoch))
+    assertLastOffsetForLeaderError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
+    assertLastOffsetForLeaderError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
+  }
+
+  @Test
+  def testOffsetForLeaderEpochValidationForFollower(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
+
+    def assertLastOffsetForLeaderError(error: Errors,
+                                       currentLeaderEpochOpt: Optional[Integer],
+                                       fetchOnlyLeader: Boolean): Unit = {
+      val endOffset = partition.lastOffsetForLeaderEpoch(currentLeaderEpochOpt, 0,
+        fetchOnlyFromLeader = fetchOnlyLeader)
+      assertEquals(error, endOffset.error)
+    }
+
+    assertLastOffsetForLeaderError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
+    assertLastOffsetForLeaderError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
+    assertLastOffsetForLeaderError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
+    assertLastOffsetForLeaderError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
+
+    assertLastOffsetForLeaderError(Errors.NOT_LEADER_FOR_PARTITION, Optional.empty(), fetchOnlyLeader = true)
+    assertLastOffsetForLeaderError(Errors.NOT_LEADER_FOR_PARTITION, Optional.of(leaderEpoch), fetchOnlyLeader = true)
+    assertLastOffsetForLeaderError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
+    assertLastOffsetForLeaderError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
+  }
+
+  @Test
+  def testReadRecordEpochValidationForLeader(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
+
+    def assertReadRecordsError(error: Errors,
+                               currentLeaderEpochOpt: Optional[Integer]): Unit = {
+      try {
+        partition.readRecords(0L, currentLeaderEpochOpt,
+          maxBytes = 1024,
+          fetchIsolation = FetchLogEnd,
+          fetchOnlyFromLeader = true,
+          minOneMessage = false)
+        if (error != Errors.NONE)
+          fail(s"Expected readRecords to fail with error $error")
+      } catch {
+        case e: Exception =>
+          assertEquals(error, Errors.forException(e))
+      }
+    }
+
+    assertReadRecordsError(Errors.NONE, Optional.empty())
+    assertReadRecordsError(Errors.NONE, Optional.of(leaderEpoch))
+    assertReadRecordsError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
+    assertReadRecordsError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
+  }
+
+  @Test
+  def testReadRecordEpochValidationForFollower(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
+
+    def assertReadRecordsError(error: Errors,
+                                       currentLeaderEpochOpt: Optional[Integer],
+                                       fetchOnlyLeader: Boolean): Unit = {
+      try {
+        partition.readRecords(0L, currentLeaderEpochOpt,
+          maxBytes = 1024,
+          fetchIsolation = FetchLogEnd,
+          fetchOnlyFromLeader = fetchOnlyLeader,
+          minOneMessage = false)
+        if (error != Errors.NONE)
+          fail(s"Expected readRecords to fail with error $error")
+      } catch {
+        case e: Exception =>
+          assertEquals(error, Errors.forException(e))
+      }
+    }
+
+    assertReadRecordsError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
+    assertReadRecordsError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
+    assertReadRecordsError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
+    assertReadRecordsError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
+
+    assertReadRecordsError(Errors.NOT_LEADER_FOR_PARTITION, Optional.empty(), fetchOnlyLeader = true)
+    assertReadRecordsError(Errors.NOT_LEADER_FOR_PARTITION, Optional.of(leaderEpoch), fetchOnlyLeader = true)
+    assertReadRecordsError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
+    assertReadRecordsError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
+  }
+
+  @Test
+  def testFetchOffsetForTimestampEpochValidationForLeader(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
+
+    def assertFetchOffsetError(error: Errors,
+                               currentLeaderEpochOpt: Optional[Integer]): Unit = {
+      try {
+        partition.fetchOffsetForTimestamp(0L,
+          isolationLevel = None,
+          currentLeaderEpoch = currentLeaderEpochOpt,
+          fetchOnlyFromLeader = true)
+        if (error != Errors.NONE)
+          fail(s"Expected readRecords to fail with error $error")
+      } catch {
+        case e: Exception =>
+          assertEquals(error, Errors.forException(e))
+      }
+    }
+
+    assertFetchOffsetError(Errors.NONE, Optional.empty())
+    assertFetchOffsetError(Errors.NONE, Optional.of(leaderEpoch))
+    assertFetchOffsetError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
+    assertFetchOffsetError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
+  }
+
+  @Test
+  def testFetchOffsetForTimestampEpochValidationForFollower(): Unit = {
+    val leaderEpoch = 5
+    val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
+
+    def assertFetchOffsetError(error: Errors,
+                               currentLeaderEpochOpt: Optional[Integer],
+                               fetchOnlyLeader: Boolean): Unit = {
+      try {
+        partition.fetchOffsetForTimestamp(0L,
+          isolationLevel = None,
+          currentLeaderEpoch = currentLeaderEpochOpt,
+          fetchOnlyFromLeader = fetchOnlyLeader)
+        if (error != Errors.NONE)
+          fail(s"Expected readRecords to fail with error $error")
+      } catch {
+        case e: Exception =>
+          assertEquals(error, Errors.forException(e))
+      }
+    }
+
+    assertFetchOffsetError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
+    assertFetchOffsetError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
+    assertFetchOffsetError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
+    assertFetchOffsetError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
+
+    assertFetchOffsetError(Errors.NOT_LEADER_FOR_PARTITION, Optional.empty(), fetchOnlyLeader = true)
+    assertFetchOffsetError(Errors.NOT_LEADER_FOR_PARTITION, Optional.of(leaderEpoch), fetchOnlyLeader = true)
+    assertFetchOffsetError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
+    assertFetchOffsetError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
   }
 
 
+  private def setupPartitionWithMocks(leaderEpoch: Int,
+                                      isLeader: Boolean,
+                                      log: Log = logManager.getOrCreateLog(topicPartition, logConfig)): Partition = {
+    val replica = new Replica(brokerId, topicPartition, time, log = Some(log))
+    val replicaManager = EasyMock.mock(classOf[ReplicaManager])
+    val zkClient = EasyMock.mock(classOf[KafkaZkClient])
+
+    val partition = new Partition(topicPartition,
+      isOffline = false,
+      replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
+      localBrokerId = brokerId,
+      time,
+      replicaManager,
+      logManager,
+      zkClient)
+
+    EasyMock.replay(replicaManager, zkClient)
+
+    partition.addReplicaIfNotExists(replica)
+
+    val controllerId = 0
+    val controllerEpoch = 0
+    val replicas = List[Integer](brokerId, brokerId + 1).asJava
+    val isr = replicas
+
+    if (isLeader) {
+      assertTrue("Expected become leader transition to succeed",
+        partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
+          leaderEpoch, isr, 1, replicas, true), 0))
+      assertEquals(leaderEpoch, partition.getLeaderEpoch)
+      assertEquals(Some(replica), partition.leaderReplicaIfLocal)
+    } else {
+      assertTrue("Expected become follower transition to succeed",
+        partition.makeFollower(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId + 1,
+          leaderEpoch, isr, 1, replicas, true), 0))
+      assertEquals(leaderEpoch, partition.getLeaderEpoch)
+      assertEquals(None, partition.leaderReplicaIfLocal)
+    }
+
+    partition
+  }
+
   @Test
   def testAppendRecordsAsFollowerBelowLogStartOffset(): Unit = {
     val log = logManager.getOrCreateLog(topicPartition, logConfig)
     val replica = new Replica(brokerId, topicPartition, time, log = Some(log))
-    val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
+    val partition = Partition(topicPartition, time, replicaManager)
     partition.addReplicaIfNotExists(replica)
-    assertEquals(Some(replica), partition.getReplica(replica.brokerId))
+    assertEquals(Some(replica), partition.localReplica)
 
     val initialLogStartOffset = 5L
     partition.truncateFullyAndStartAt(initialLogStartOffset, isFuture = false)
@@ -226,24 +462,95 @@ class PartitionTest {
   }
 
   @Test
+  def testListOffsetIsolationLevels(): Unit = {
+    val log = logManager.getOrCreateLog(topicPartition, logConfig)
+    val replica = new Replica(brokerId, topicPartition, time, log = Some(log))
+    val replicaManager = EasyMock.mock(classOf[ReplicaManager])
+    val zkClient = EasyMock.mock(classOf[KafkaZkClient])
+
+    val partition = new Partition(topicPartition,
+      isOffline = false,
+      replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
+      localBrokerId = brokerId,
+      time,
+      replicaManager,
+      logManager,
+      zkClient)
+
+    val controllerId = 0
+    val controllerEpoch = 0
+    val leaderEpoch = 5
+    val replicas = List[Integer](brokerId, brokerId + 1).asJava
+    val isr = replicas
+
+    EasyMock.expect(replicaManager.tryCompleteDelayedFetch(EasyMock.anyObject[TopicPartitionOperationKey]))
+        .andVoid()
+
+    EasyMock.replay(replicaManager, zkClient)
+
+    partition.addReplicaIfNotExists(replica)
+
+    assertTrue("Expected become leader transition to succeed",
+      partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId,
+        leaderEpoch, isr, 1, replicas, true), 0))
+    assertEquals(leaderEpoch, partition.getLeaderEpoch)
+    assertEquals(Some(replica), partition.leaderReplicaIfLocal)
+
+    val records = createTransactionalRecords(List(
+      new SimpleRecord("k1".getBytes, "v1".getBytes),
+      new SimpleRecord("k2".getBytes, "v2".getBytes),
+      new SimpleRecord("k3".getBytes, "v3".getBytes)),
+      baseOffset = 0L)
+    partition.appendRecordsToLeader(records, isFromClient = true)
+
+    def fetchLatestOffset(isolationLevel: Option[IsolationLevel]): TimestampOffset = {
+      partition.fetchOffsetForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP,
+        isolationLevel = isolationLevel,
+        currentLeaderEpoch = Optional.empty(),
+        fetchOnlyFromLeader = true)
+    }
+
+    def fetchEarliestOffset(isolationLevel: Option[IsolationLevel]): TimestampOffset = {
+      partition.fetchOffsetForTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP,
+        isolationLevel = isolationLevel,
+        currentLeaderEpoch = Optional.empty(),
+        fetchOnlyFromLeader = true)
+    }
+
+    assertEquals(3L, fetchLatestOffset(isolationLevel = None).offset)
+    assertEquals(0L, fetchLatestOffset(isolationLevel = Some(IsolationLevel.READ_UNCOMMITTED)).offset)
+    assertEquals(0L, fetchLatestOffset(isolationLevel = Some(IsolationLevel.READ_COMMITTED)).offset)
+
+    replica.highWatermark = LogOffsetMetadata(1L)
+
+    assertEquals(3L, fetchLatestOffset(isolationLevel = None).offset)
+    assertEquals(1L, fetchLatestOffset(isolationLevel = Some(IsolationLevel.READ_UNCOMMITTED)).offset)
+    assertEquals(0L, fetchLatestOffset(isolationLevel = Some(IsolationLevel.READ_COMMITTED)).offset)
+
+    assertEquals(0L, fetchEarliestOffset(isolationLevel = None).offset)
+    assertEquals(0L, fetchEarliestOffset(isolationLevel = Some(IsolationLevel.READ_UNCOMMITTED)).offset)
+    assertEquals(0L, fetchEarliestOffset(isolationLevel = Some(IsolationLevel.READ_COMMITTED)).offset)
+  }
+
+  @Test
   def testGetReplica(): Unit = {
     val log = logManager.getOrCreateLog(topicPartition, logConfig)
     val replica = new Replica(brokerId, topicPartition, time, log = Some(log))
-    val partition = new
-        Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
+    val partition = Partition(topicPartition, time, replicaManager)
 
-    assertEquals(None, partition.getReplica(brokerId))
+    assertEquals(None, partition.localReplica)
     assertThrows[ReplicaNotAvailableException] {
-      partition.getReplicaOrException(brokerId)
+      partition.localReplicaOrException
     }
 
     partition.addReplicaIfNotExists(replica)
-    assertEquals(replica, partition.getReplicaOrException(brokerId))
+    assertEquals(Some(replica), partition.localReplica)
+    assertEquals(replica, partition.localReplicaOrException)
   }
 
   @Test
   def testAppendRecordsToFollowerWithNoReplicaThrowsException(): Unit = {
-    val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
+    val partition = Partition(topicPartition, time, replicaManager)
     assertThrows[ReplicaNotAvailableException] {
       partition.appendRecordsToFollowerOrFutureReplica(
            createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 0L), isFuture = false)
@@ -252,22 +559,21 @@ class PartitionTest {
 
   @Test
   def testMakeFollowerWithNoLeaderIdChange(): Unit = {
-    val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
+    val partition = Partition(topicPartition, time, replicaManager)
 
     // Start off as follower
-    var partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 1, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
+    var partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 1,
+      List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
     partition.makeFollower(0, partitionStateInfo, 0)
 
-    // Request with same leader and epoch increases by more than 1, perform become-follower steps
-    partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 3, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
-    assertTrue(partition.makeFollower(0, partitionStateInfo, 1))
-
-    // Request with same leader and epoch increases by only 1, skip become-follower steps
-    partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
-    assertFalse(partition.makeFollower(0, partitionStateInfo, 2))
+    // Request with same leader and epoch increases by only 1, do become-follower steps
+    partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4,
+      List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
+    assertTrue(partition.makeFollower(0, partitionStateInfo, 2))
 
     // Request with same leader and same epoch, skip become-follower steps
-    partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
+    partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4,
+      List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false)
     assertFalse(partition.makeFollower(0, partitionStateInfo, 2))
   }
 
@@ -289,7 +595,7 @@ class PartitionTest {
     val batch3 = TestUtils.records(records = List(new SimpleRecord("k6".getBytes, "v1".getBytes),
                                                   new SimpleRecord("k7".getBytes, "v2".getBytes)))
 
-    val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager)
+    val partition = Partition(topicPartition, time, replicaManager)
     assertTrue("Expected first makeLeader() to return 'leader changed'",
                partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 1, replicas, true), 0))
     assertEquals("Current leader epoch", leaderEpoch, partition.getLeaderEpoch)
@@ -356,4 +662,18 @@ class PartitionTest {
     builder.build()
   }
 
+  def createTransactionalRecords(records: Iterable[SimpleRecord],
+                                 baseOffset: Long,
+                                 partitionLeaderEpoch: Int = 0): MemoryRecords = {
+    val producerId = 1L
+    val producerEpoch = 0.toShort
+    val baseSequence = 0
+    val isTransactional = true
+    val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
+    val builder = MemoryRecords.builder(buf, CompressionType.NONE, baseOffset, producerId,
+      producerEpoch, baseSequence, isTransactional)
+    records.foreach(builder.append)
+    builder.build()
+  }
+
 }
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
index f648257..e10c3ea 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
@@ -1902,8 +1902,11 @@ class GroupMetadataManagerTest {
     val fileRecordsMock = EasyMock.mock(classOf[FileRecords])
 
     EasyMock.expect(logMock.logStartOffset).andStubReturn(startOffset)
-    EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None),
-      EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED)))
+    EasyMock.expect(logMock.read(EasyMock.eq(startOffset),
+      maxLength = EasyMock.anyInt(),
+      maxOffset = EasyMock.eq(None),
+      minOneMessage = EasyMock.eq(true),
+      includeAbortedTxns = EasyMock.eq(false)))
       .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock))
 
     EasyMock.expect(fileRecordsMock.sizeInBytes()).andStubReturn(records.sizeInBytes)
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala
index 060e07e..b2a6733 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala
@@ -255,8 +255,11 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren
     val endOffset = startOffset + records.records.asScala.size
 
     EasyMock.expect(logMock.logStartOffset).andStubReturn(startOffset)
-    EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None),
-      EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED)))
+    EasyMock.expect(logMock.read(EasyMock.eq(startOffset),
+      maxLength = EasyMock.anyInt(),
+      maxOffset = EasyMock.eq(None),
+      minOneMessage = EasyMock.eq(true),
+      includeAbortedTxns = EasyMock.eq(false)))
       .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock))
 
     EasyMock.expect(fileRecordsMock.sizeInBytes()).andStubReturn(records.sizeInBytes)
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
index 74bbe33..b395d00 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
@@ -582,8 +582,11 @@ class TransactionStateManagerTest {
     EasyMock.expect(replicaManager.getLogEndOffset(topicPartition)).andStubReturn(Some(endOffset))
 
     EasyMock.expect(logMock.logStartOffset).andStubReturn(startOffset)
-    EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None),
-      EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED)))
+    EasyMock.expect(logMock.read(EasyMock.eq(startOffset),
+      maxLength = EasyMock.anyInt(),
+      maxOffset = EasyMock.eq(None),
+      minOneMessage = EasyMock.eq(true),
+      includeAbortedTxns = EasyMock.eq(false)))
       .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock))
 
     EasyMock.expect(fileRecordsMock.sizeInBytes()).andStubReturn(records.sizeInBytes)
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index 0c97357..33001fd 100755
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -112,7 +112,7 @@ abstract class KafkaServerTestHarness extends ZooKeeperTestHarness {
     if (servers != null) {
       TestUtils.shutdownServers(servers)
     }
-    super.tearDown
+    super.tearDown()
   }
 
   /**
diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
index 1cf393e..8372387 100755
--- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
@@ -25,7 +25,7 @@ import org.junit.Assert._
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.runners.Parameterized.Parameters
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.utils.Utils
 import java.util.{Collection, Properties}
 
@@ -64,7 +64,11 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
     log.appendAsLeader(MemoryRecords.withRecords(CompressionType.forId(messageCompressionCode.codec), 0,
           new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
 
-    def readBatch(offset: Int) = log.readUncommitted(offset, 4096).records.batches.iterator.next()
+    def readBatch(offset: Int): RecordBatch = {
+      val fetchInfo = log.read(offset, 4096, maxOffset = None,
+        includeAbortedTxns = false, minOneMessage = true)
+      fetchInfo.records.batches.iterator.next()
+    }
 
     if (!brokerCompression.equals("producer")) {
       val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression)
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index ae8bc01..2fbb875 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -20,6 +20,7 @@ package kafka.log
 import java.io._
 import java.util.Properties
 
+import kafka.server.FetchDataInfo
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.utils._
 import org.apache.kafka.common.{KafkaException, TopicPartition}
@@ -128,10 +129,10 @@ class LogManagerTest {
 
     // there should be a log file, two indexes, one producer snapshot, and the leader epoch checkpoint
     assertEquals("Files should have been deleted", log.numberOfSegments * 4 + 1, log.dir.list.length)
-    assertEquals("Should get empty fetch off new log.", 0, log.readUncommitted(offset+1, 1024).records.sizeInBytes)
+    assertEquals("Should get empty fetch off new log.", 0, readLog(log, offset + 1).records.sizeInBytes)
 
     try {
-      log.readUncommitted(0, 1024)
+      readLog(log, 0)
       fail("Should get exception from fetching earlier.")
     } catch {
       case _: OffsetOutOfRangeException => // This is good.
@@ -178,9 +179,9 @@ class LogManagerTest {
     // there should be a log file, two indexes (the txn index is created lazily),
     // the leader epoch checkpoint and two producer snapshot files (one for the active and previous segments)
     assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 3, log.dir.list.length)
-    assertEquals("Should get empty fetch off new log.", 0, log.readUncommitted(offset + 1, 1024).records.sizeInBytes)
+    assertEquals("Should get empty fetch off new log.", 0, readLog(log, offset + 1).records.sizeInBytes)
     try {
-      log.readUncommitted(0, 1024)
+      readLog(log, 0)
       fail("Should get exception from fetching earlier.")
     } catch {
       case _: OffsetOutOfRangeException => // This is good.
@@ -373,4 +374,9 @@ class LogManagerTest {
     time.sleep(logManager.currentDefaultConfig.fileDeleteDelayMs - logManager.InitialTaskDelayMs)
     assertFalse("Logs not deleted", logManager.hasLogsToBeDeleted)
   }
+
+  private def readLog(log: Log, offset: Long, maxLength: Int = 1024): FetchDataInfo = {
+    log.read(offset, maxLength, maxOffset = None, minOneMessage = true, includeAbortedTxns = false)
+  }
+
 }
diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala
index f8d76b6..151c4ed 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -34,7 +34,6 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter
 import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
-import org.apache.kafka.common.requests.IsolationLevel
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.easymock.EasyMock
 import org.junit.Assert._
@@ -1055,7 +1054,7 @@ class LogTest {
     log.appendAsFollower(memoryRecords)
     log.flush()
 
-    val fetchedData = log.readUncommitted(0, Int.MaxValue)
+    val fetchedData = readLog(log, 0, Int.MaxValue)
 
     val origIterator = memoryRecords.batches.iterator()
     for (batch <- fetchedData.records.batches.asScala) {
@@ -1222,18 +1221,18 @@ class LogTest {
       log.appendAsLeader(TestUtils.singletonRecords(value = value), leaderEpoch = 0)
 
     for(i <- values.indices) {
-      val read = log.readUncommitted(i, 100, Some(i+1)).records.batches.iterator.next()
+      val read = readLog(log, i, 100, Some(i+1)).records.batches.iterator.next()
       assertEquals("Offset read should match order appended.", i, read.lastOffset)
       val actual = read.iterator.next()
       assertNull("Key should be null", actual.key)
       assertEquals("Values not equal", ByteBuffer.wrap(values(i)), actual.value)
     }
     assertEquals("Reading beyond the last message returns nothing.", 0,
-      log.readUncommitted(values.length, 100, None).records.batches.asScala.size)
+      readLog(log, values.length, 100).records.batches.asScala.size)
   }
 
   /**
-   * This test appends a bunch of messages with non-sequential offsets and checks that we can read the correct message
+   * This test appends a bunch of messages with non-sequential offsets and checks that we can an the correct message
    * from any offset less than the logEndOffset including offsets not appended.
    */
   @Test
@@ -1248,7 +1247,7 @@ class LogTest {
       log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i)))
     for(i <- 50 until messageIds.max) {
       val idx = messageIds.indexWhere(_ >= i)
-      val read = log.readUncommitted(i, 100, None).records.records.iterator.next()
+      val read = readLog(log, i, 100).records.records.iterator.next()
       assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
       assertEquals("Message should match appended.", records(idx), new SimpleRecord(read))
     }
@@ -1273,7 +1272,7 @@ class LogTest {
     log.logSegments.head.truncateTo(1)
 
     assertEquals("A read should now return the last message in the log", log.logEndOffset - 1,
-      log.readUncommitted(1, 200, None).records.batches.iterator.next().lastOffset)
+      readLog(log, 1, 200).records.batches.iterator.next().lastOffset)
   }
 
   @Test(expected = classOf[KafkaStorageException])
@@ -1298,16 +1297,17 @@ class LogTest {
     for (i <- 50 until messageIds.max) {
       val idx = messageIds.indexWhere(_ >= i)
       val reads = Seq(
-        log.readUncommitted(i, 1, minOneMessage = true),
-        log.readUncommitted(i, 100, minOneMessage = true),
-        log.readUncommitted(i, 100, Some(10000), minOneMessage = true)
+        readLog(log, i, 1),
+        readLog(log, i, 100),
+        readLog(log, i, 100, Some(10000))
       ).map(_.records.records.iterator.next())
       reads.foreach { read =>
         assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
         assertEquals("Message should match appended.", records(idx), new SimpleRecord(read))
       }
 
-      assertEquals(Seq.empty, log.readUncommitted(i, 1, Some(1), minOneMessage = true).records.batches.asScala.toIndexedSeq)
+      val fetchedData = readLog(log, i, 1, Some(1))
+      assertEquals(Seq.empty, fetchedData.records.batches.asScala.toIndexedSeq)
     }
   }
 
@@ -1323,14 +1323,14 @@ class LogTest {
       log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i)))
 
     for (i <- 50 until messageIds.max) {
-      assertEquals(MemoryRecords.EMPTY, log.readUncommitted(i, 0).records)
+      assertEquals(MemoryRecords.EMPTY, readLog(log, i, maxLength = 0, minOneMessage = false).records)
 
       // we return an incomplete message instead of an empty one for the case below
       // we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is
       // larger than the fetch size
       // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty
       // partition
-      val fetchInfo = log.readUncommitted(i, 1)
+      val fetchInfo = readLog(log, i, maxLength = 1, minOneMessage = false)
       assertTrue(fetchInfo.firstEntryIncomplete)
       assertTrue(fetchInfo.records.isInstanceOf[FileRecords])
       assertEquals(1, fetchInfo.records.sizeInBytes)
@@ -1352,24 +1352,24 @@ class LogTest {
     log.appendAsLeader(TestUtils.singletonRecords(value = "42".getBytes), leaderEpoch = 0)
 
     assertEquals("Reading at the log end offset should produce 0 byte read.", 0,
-      log.readUncommitted(1025, 1000).records.sizeInBytes)
+      readLog(log, 1025, 1000).records.sizeInBytes)
 
     try {
-      log.readUncommitted(0, 1000)
+      readLog(log, 0, 1000)
       fail("Reading below the log start offset should throw OffsetOutOfRangeException")
     } catch {
       case _: OffsetOutOfRangeException => // This is good.
     }
 
     try {
-      log.readUncommitted(1026, 1000)
+      readLog(log, 1026, 1000)
       fail("Reading at beyond the log end offset should throw OffsetOutOfRangeException")
     } catch {
       case _: OffsetOutOfRangeException => // This is good.
     }
 
     assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0,
-      log.readUncommitted(1025, 1000, Some(1024)).records.sizeInBytes)
+      readLog(log, 1025, 1000, Some(1024)).records.sizeInBytes)
   }
 
   /**
@@ -1390,7 +1390,7 @@ class LogTest {
     /* do successive reads to ensure all our messages are there */
     var offset = 0L
     for(i <- 0 until numMessages) {
-      val messages = log.readUncommitted(offset, 1024*1024).records.batches
+      val messages = readLog(log, offset, 1024*1024).records.batches
       val head = messages.iterator.next()
       assertEquals("Offsets not equal", offset, head.lastOffset)
 
@@ -1401,7 +1401,7 @@ class LogTest {
       assertEquals(s"Timestamps not equal at offset $offset", expected.timestamp, actual.timestamp)
       offset = head.lastOffset + 1
     }
-    val lastRead = log.readUncommitted(startOffset = numMessages, maxLength = 1024*1024,
+    val lastRead = readLog(log, startOffset = numMessages, maxLength = 1024*1024,
       maxOffset = Some(numMessages + 1)).records
     assertEquals("Should be no more messages", 0, lastRead.records.asScala.size)
 
@@ -1424,7 +1424,7 @@ class LogTest {
     log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
     log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0)
 
-    def read(offset: Int) = log.readUncommitted(offset, 4096).records.records
+    def read(offset: Int) = readLog(log, offset, 4096).records.records
 
     /* we should always get the first message in the compressed set when reading any offset in the set */
     assertEquals("Read at offset 0 should produce 0", 0, read(0).iterator.next().offset)
@@ -1645,7 +1645,7 @@ class LogTest {
     assertTrue("The index should have been rebuilt", log.logSegments.head.offsetIndex.entries > 0)
     assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
     for(i <- 0 until numMessages) {
-      assertEquals(i, log.readUncommitted(i, 100, None).records.batches.iterator.next().lastOffset)
+      assertEquals(i, readLog(log, i, 100).records.batches.iterator.next().lastOffset)
       if (i == 0)
         assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(mockTime.milliseconds + i * 10).get.offset)
       else
@@ -1713,7 +1713,7 @@ class LogTest {
     log = createLog(logDir, logConfig, recoveryPoint = 200L)
     assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
     for(i <- 0 until numMessages) {
-      assertEquals(i, log.readUncommitted(i, 100, None).records.batches.iterator.next().lastOffset)
+      assertEquals(i, readLog(log, i, 100).records.batches.iterator.next().lastOffset)
       if (i == 0)
         assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(mockTime.milliseconds + i * 10).get.offset)
       else
@@ -1919,7 +1919,7 @@ class LogTest {
   def testAppendMessageWithNullPayload() {
     val log = createLog(logDir, LogConfig())
     log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0)
-    val head = log.readUncommitted(0, 4096, None).records.records.iterator.next()
+    val head = readLog(log, 0, 4096).records.records.iterator.next()
     assertEquals(0, head.offset)
     assertTrue("Message payload should be null.", !head.hasValue)
   }
@@ -2790,7 +2790,7 @@ class LogTest {
 
     //Then leader epoch should be set on messages
     for (i <- records.indices) {
-      val read = log.readUncommitted(i, 100, Some(i+1)).records.batches.iterator.next()
+      val read = readLog(log, i, 100, Some(i+1)).records.batches.iterator.next()
       assertEquals("Should have set leader epoch", 72, read.partitionLeaderEpoch)
     }
   }
@@ -3421,7 +3421,7 @@ class LogTest {
     assertEquals(None, log.firstUnstableOffset.map(_.messageOffset))
 
     // now check that a fetch includes the aborted transaction
-    val fetchDataInfo = log.read(0L, 2048, isolationLevel = IsolationLevel.READ_COMMITTED)
+    val fetchDataInfo = log.read(0L, 2048, maxOffset = None, minOneMessage = true, includeAbortedTxns = true)
     assertEquals(1, fetchDataInfo.abortedTransactions.size)
 
     assertTrue(fetchDataInfo.abortedTransactions.isDefined)
@@ -3538,6 +3538,12 @@ class LogTest {
       expectDeletedFiles)
   }
 
+  private def readLog(log: Log, startOffset: Long, maxLength: Int,
+                      maxOffset: Option[Long] = None,
+                      minOneMessage: Boolean = true): FetchDataInfo = {
+    log.read(startOffset, maxLength, maxOffset, minOneMessage, includeAbortedTxns = false)
+  }
+
 }
 
 object LogTest {
diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
new file mode 100644
index 0000000..cd00ff1
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.server
+
+import kafka.cluster.BrokerEndPoint
+import org.apache.kafka.common.TopicPartition
+import org.easymock.EasyMock
+import org.junit.Test
+import org.junit.Assert._
+
+class AbstractFetcherManagerTest {
+
+  @Test
+  def testAddAndRemovePartition(): Unit = {
+    val fetcher = EasyMock.mock(classOf[AbstractFetcherThread])
+    val fetcherManager = new AbstractFetcherManager[AbstractFetcherThread]("fetcher-manager", "fetcher-manager", 2) {
+      override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = {
+        fetcher
+      }
+    }
+
+    val fetchOffset = 10L
+    val leaderEpoch = 15
+    val tp = new TopicPartition("topic", 0)
+    val initialFetchState = InitialFetchState(
+      leader = new BrokerEndPoint(0, "localhost", 9092),
+      currentLeaderEpoch = leaderEpoch,
+      initOffset = fetchOffset)
+
+    EasyMock.expect(fetcher.start())
+    EasyMock.expect(fetcher.addPartitions(Map(tp -> OffsetAndEpoch(fetchOffset, leaderEpoch))))
+    EasyMock.expect(fetcher.fetchState(tp))
+      .andReturn(Some(PartitionFetchState(fetchOffset, leaderEpoch, Truncating)))
+    EasyMock.expect(fetcher.removePartitions(Set(tp)))
+    EasyMock.expect(fetcher.fetchState(tp)).andReturn(None)
+    EasyMock.replay(fetcher)
+
+    fetcherManager.addFetcherForPartitions(Map(tp -> initialFetchState))
+    assertEquals(Some(fetcher), fetcherManager.getFetcher(tp))
+
+    fetcherManager.removeFetcherForPartitions(Set(tp))
+    assertEquals(None, fetcherManager.getFetcher(tp))
+
+    EasyMock.verify(fetcher)
+  }
+
+}
diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index 7a7aeb3..77ba934 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -19,15 +19,16 @@ package kafka.server
 
 import java.nio.ByteBuffer
 import java.util.Optional
+import java.util.concurrent.atomic.AtomicInteger
 
-import AbstractFetcherThread._
 import com.yammer.metrics.Metrics
 import kafka.cluster.BrokerEndPoint
 import kafka.log.LogAppendInfo
 import kafka.message.NoCompressionCodec
+import kafka.server.AbstractFetcherThread.ResultWithPartitions
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.internals.FatalExitError
+import org.apache.kafka.common.errors.{FencedLeaderEpochException, UnknownLeaderEpochException}
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.{EpochEndOffset, FetchRequest}
@@ -54,15 +55,19 @@ class AbstractFetcherThreadTest {
       .batches.asScala.head
   }
 
+  private def offsetAndEpoch(fetchOffset: Long, leaderEpoch: Int): OffsetAndEpoch = {
+    OffsetAndEpoch(offset = fetchOffset, leaderEpoch = leaderEpoch)
+  }
+
   @Test
   def testMetricsRemovedOnShutdown(): Unit = {
     val partition = new TopicPartition("topic", 0)
     val fetcher = new MockFetcherThread
 
     // add one partition to create the consumer lag metric
-    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState())
-    fetcher.addPartitions(Map(partition -> 0L))
-    fetcher.setLeaderState(partition, MockFetcherThread.PartitionState())
+    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 0)))
+    fetcher.setLeaderState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
 
     fetcher.start()
 
@@ -83,9 +88,9 @@ class AbstractFetcherThreadTest {
     val fetcher = new MockFetcherThread
 
     // add one partition to create the consumer lag metric
-    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState())
-    fetcher.addPartitions(Map(partition -> 0L))
-    fetcher.setLeaderState(partition, MockFetcherThread.PartitionState())
+    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 0)))
+    fetcher.setLeaderState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
 
     fetcher.doWork()
 
@@ -104,12 +109,12 @@ class AbstractFetcherThreadTest {
     val partition = new TopicPartition("topic", 0)
     val fetcher = new MockFetcherThread
 
-    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState())
-    fetcher.addPartitions(Map(partition -> 0L))
+    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 0)))
 
     val batch = mkBatch(baseOffset = 0L, leaderEpoch = 0,
       new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))
-    val leaderState = MockFetcherThread.PartitionState(Seq(batch), highWatermark = 2L)
+    val leaderState = MockFetcherThread.PartitionState(Seq(batch), leaderEpoch = 0, highWatermark = 2L)
     fetcher.setLeaderState(partition, leaderState)
 
     fetcher.doWork()
@@ -120,6 +125,129 @@ class AbstractFetcherThreadTest {
   }
 
   @Test
+  def testFencedTruncation(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+    val fetcher = new MockFetcherThread
+
+    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 0)))
+
+    val batch = mkBatch(baseOffset = 0L, leaderEpoch = 1,
+      new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))
+    val leaderState = MockFetcherThread.PartitionState(Seq(batch), leaderEpoch = 1, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    fetcher.doWork()
+
+    // No progress should be made
+    val replicaState = fetcher.replicaPartitionState(partition)
+    assertEquals(0L, replicaState.logEndOffset)
+    assertEquals(0L, replicaState.highWatermark)
+
+    // After fencing, the fetcher should remove the partition from tracking
+    assertTrue(fetcher.fetchState(partition).isEmpty)
+  }
+
+  @Test
+  def testFencedFetch(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+    val fetcher = new MockFetcherThread
+
+    val replicaState = MockFetcherThread.PartitionState(leaderEpoch = 0)
+    fetcher.setReplicaState(partition, replicaState)
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 0)))
+
+    val batch = mkBatch(baseOffset = 0L, leaderEpoch = 0,
+      new SimpleRecord("a".getBytes),
+      new SimpleRecord("b".getBytes))
+    val leaderState = MockFetcherThread.PartitionState(Seq(batch), leaderEpoch = 0, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    fetcher.doWork()
+
+    // Verify we have caught up
+    assertEquals(2, replicaState.logEndOffset)
+
+    // Bump the epoch on the leader
+    fetcher.leaderPartitionState(partition).leaderEpoch += 1
+
+    fetcher.doWork()
+
+    // After fencing, the fetcher should remove the partition from tracking
+    assertTrue(fetcher.fetchState(partition).isEmpty)
+  }
+
+  @Test
+  def testUnknownLeaderEpochInTruncation(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+    val fetcher = new MockFetcherThread
+
+    // The replica's leader epoch is ahead of the leader
+    val replicaState = MockFetcherThread.PartitionState(leaderEpoch = 1)
+    fetcher.setReplicaState(partition, replicaState)
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 1)))
+
+    val batch = mkBatch(baseOffset = 0L, leaderEpoch = 0, new SimpleRecord("a".getBytes))
+    val leaderState = MockFetcherThread.PartitionState(Seq(batch), leaderEpoch = 0, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    fetcher.doWork()
+
+    // Not data has been fetched and the follower is still truncating
+    assertEquals(0, replicaState.logEndOffset)
+    assertEquals(Some(Truncating), fetcher.fetchState(partition).map(_.state))
+
+    // Bump the epoch on the leader
+    fetcher.leaderPartitionState(partition).leaderEpoch += 1
+
+    // Now we can make progress
+    fetcher.doWork()
+
+    assertEquals(1, replicaState.logEndOffset)
+    assertEquals(Some(Fetching), fetcher.fetchState(partition).map(_.state))
+  }
+
+  @Test
+  def testUnknownLeaderEpochWhileFetching(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+    val fetcher = new MockFetcherThread
+
+    // This test is contrived because it shouldn't be possible to to see unknown leader epoch
+    // in the Fetching state as the leader must validate the follower's epoch when it checks
+    // the truncation offset.
+
+    val replicaState = MockFetcherThread.PartitionState(leaderEpoch = 1)
+    fetcher.setReplicaState(partition, replicaState)
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 1)))
+
+    val leaderState = MockFetcherThread.PartitionState(Seq(
+      mkBatch(baseOffset = 0L, leaderEpoch = 0, new SimpleRecord("a".getBytes)),
+      mkBatch(baseOffset = 1L, leaderEpoch = 0, new SimpleRecord("b".getBytes))
+    ), leaderEpoch = 1, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    fetcher.doWork()
+
+    // We have fetched one batch and gotten out of the truncation phase
+    assertEquals(1, replicaState.logEndOffset)
+    assertEquals(Some(Fetching), fetcher.fetchState(partition).map(_.state))
+
+    // Somehow the leader epoch rewinds
+    fetcher.leaderPartitionState(partition).leaderEpoch = 0
+
+    // We are stuck at the current offset
+    fetcher.doWork()
+    assertEquals(1, replicaState.logEndOffset)
+    assertEquals(Some(Fetching), fetcher.fetchState(partition).map(_.state))
+
+    // After returning to the right epoch, we can continue fetching
+    fetcher.leaderPartitionState(partition).leaderEpoch = 1
+    fetcher.doWork()
+    assertEquals(2, replicaState.logEndOffset)
+    assertEquals(Some(Fetching), fetcher.fetchState(partition).map(_.state))
+  }
+
+  @Test
   def testTruncation(): Unit = {
     val partition = new TopicPartition("topic", 0)
     val fetcher = new MockFetcherThread
@@ -129,16 +257,16 @@ class AbstractFetcherThreadTest {
       mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
       mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
-    val replicaState = MockFetcherThread.PartitionState(replicaLog, highWatermark = 0L)
+    val replicaState = MockFetcherThread.PartitionState(replicaLog, leaderEpoch = 5, highWatermark = 0L)
     fetcher.setReplicaState(partition, replicaState)
-    fetcher.addPartitions(Map(partition -> 3L))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(3L, leaderEpoch = 5)))
 
     val leaderLog = Seq(
       mkBatch(baseOffset = 0, leaderEpoch = 1, new SimpleRecord("a".getBytes)),
       mkBatch(baseOffset = 1, leaderEpoch = 3, new SimpleRecord("b".getBytes)),
       mkBatch(baseOffset = 2, leaderEpoch = 5, new SimpleRecord("c".getBytes)))
 
-    val leaderState = MockFetcherThread.PartitionState(leaderLog, highWatermark = 2L)
+    val leaderState = MockFetcherThread.PartitionState(leaderLog, leaderEpoch = 5, highWatermark = 2L)
     fetcher.setLeaderState(partition, leaderState)
 
     TestUtils.waitUntilTrue(() => {
@@ -152,6 +280,48 @@ class AbstractFetcherThreadTest {
   }
 
   @Test
+  def testTruncationSkippedIfNoEpochChange(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+
+    var truncations = 0
+    val fetcher = new MockFetcherThread {
+      override def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = {
+        truncations += 1
+        super.truncate(topicPartition, truncationState)
+      }
+    }
+
+    val replicaState = MockFetcherThread.PartitionState(leaderEpoch = 5)
+    fetcher.setReplicaState(partition, replicaState)
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 5)))
+
+    val leaderLog = Seq(
+      mkBatch(baseOffset = 0, leaderEpoch = 1, new SimpleRecord("a".getBytes)),
+      mkBatch(baseOffset = 1, leaderEpoch = 3, new SimpleRecord("b".getBytes)),
+      mkBatch(baseOffset = 2, leaderEpoch = 5, new SimpleRecord("c".getBytes)))
+
+    val leaderState = MockFetcherThread.PartitionState(leaderLog, leaderEpoch = 5, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    // Do one round of truncation
+    fetcher.doWork()
+
+    // We only fetch one record at a time with mock fetcher
+    assertEquals(1, replicaState.logEndOffset)
+    assertEquals(1, truncations)
+
+    // Add partitions again with the same epoch
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(3L, leaderEpoch = 5)))
+
+    // Verify we did not truncate
+    fetcher.doWork()
+
+    // No truncations occurred and we have fetched another record
+    assertEquals(1, truncations)
+    assertEquals(2, replicaState.logEndOffset)
+  }
+
+  @Test
   def testFollowerFetchOutOfRangeHigh(): Unit = {
     val partition = new TopicPartition("topic", 0)
     val fetcher = new MockFetcherThread()
@@ -161,22 +331,22 @@ class AbstractFetcherThreadTest {
       mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
       mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
-    val replicaState = MockFetcherThread.PartitionState(replicaLog, highWatermark = 0L)
+    val replicaState = MockFetcherThread.PartitionState(replicaLog, leaderEpoch = 4, highWatermark = 0L)
     fetcher.setReplicaState(partition, replicaState)
-    fetcher.addPartitions(Map(partition -> 3L))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(3L, leaderEpoch = 4)))
 
     val leaderLog = Seq(
       mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)),
       mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
       mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
-    val leaderState = MockFetcherThread.PartitionState(leaderLog, highWatermark = 2L)
+    val leaderState = MockFetcherThread.PartitionState(leaderLog, leaderEpoch = 4, highWatermark = 2L)
     fetcher.setLeaderState(partition, leaderState)
 
     // initial truncation and verify that the log end offset is updated
     fetcher.doWork()
     assertEquals(3L, replicaState.logEndOffset)
-    assertFalse(fetcher.partitionStates.stateValue(partition).isTruncatingLog)
+    assertEquals(Option(Fetching), fetcher.fetchState(partition).map(_.state))
 
     // To hit this case, we have to change the leader log without going through the truncation phase
     leaderState.log.clear()
@@ -192,6 +362,35 @@ class AbstractFetcherThreadTest {
   }
 
   @Test
+  def testFencedOffsetResetAfterOutOfRange(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+    var fetchedEarliestOffset = false
+    val fetcher = new MockFetcherThread() {
+      override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, leaderEpoch: Int): Long = {
+        fetchedEarliestOffset = true
+        throw new FencedLeaderEpochException(s"Epoch $leaderEpoch is fenced")
+      }
+    }
+
+    val replicaLog = Seq()
+    val replicaState = MockFetcherThread.PartitionState(replicaLog, leaderEpoch = 4, highWatermark = 0L)
+    fetcher.setReplicaState(partition, replicaState)
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 4)))
+
+    val leaderLog = Seq(
+      mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
+      mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
+    val leaderState = MockFetcherThread.PartitionState(leaderLog, leaderEpoch = 4, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    // After the out of range error, we get a fenced error and remove the partition
+    fetcher.doWork()
+    assertEquals(0, replicaState.logEndOffset)
+    assertTrue(fetchedEarliestOffset)
+    assertTrue(fetcher.fetchState(partition).isEmpty)
+  }
+
+  @Test
   def testFollowerFetchOutOfRangeLow(): Unit = {
     val partition = new TopicPartition("topic", 0)
     val fetcher = new MockFetcherThread
@@ -200,19 +399,19 @@ class AbstractFetcherThreadTest {
     val replicaLog = Seq(
       mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)))
 
-    val replicaState = MockFetcherThread.PartitionState(replicaLog, highWatermark = 0L)
+    val replicaState = MockFetcherThread.PartitionState(replicaLog, leaderEpoch = 0, highWatermark = 0L)
     fetcher.setReplicaState(partition, replicaState)
-    fetcher.addPartitions(Map(partition -> 3L))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(3L, leaderEpoch = 0)))
 
     val leaderLog = Seq(
       mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
-    val leaderState = MockFetcherThread.PartitionState(leaderLog, highWatermark = 2L)
+    val leaderState = MockFetcherThread.PartitionState(leaderLog, leaderEpoch = 0, highWatermark = 2L)
     fetcher.setLeaderState(partition, leaderState)
 
     // initial truncation and verify that the log start offset is updated
     fetcher.doWork()
-    assertFalse(fetcher.partitionStates.stateValue(partition).isTruncatingLog)
+    assertEquals(Option(Fetching), fetcher.fetchState(partition).map(_.state))
     assertEquals(2, replicaState.logStartOffset)
     assertEquals(List(), replicaState.log.toList)
 
@@ -227,12 +426,52 @@ class AbstractFetcherThreadTest {
   }
 
   @Test
+  def testRetryAfterUnknownLeaderEpochInLatestOffsetFetch(): Unit = {
+    val partition = new TopicPartition("topic", 0)
+    val fetcher: MockFetcherThread = new MockFetcherThread {
+      val tries = new AtomicInteger(0)
+      override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, leaderEpoch: Int): Long = {
+        if (tries.getAndIncrement() == 0)
+          throw new UnknownLeaderEpochException("Unexpected leader epoch")
+        super.fetchLatestOffsetFromLeader(topicPartition, leaderEpoch)
+      }
+    }
+
+    // The follower begins from an offset which is behind the leader's log start offset
+    val replicaLog = Seq(
+      mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)))
+
+    val replicaState = MockFetcherThread.PartitionState(replicaLog, leaderEpoch = 0, highWatermark = 0L)
+    fetcher.setReplicaState(partition, replicaState)
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(3L, leaderEpoch = 0)))
+
+    val leaderLog = Seq(
+      mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
+
+    val leaderState = MockFetcherThread.PartitionState(leaderLog, leaderEpoch = 0, highWatermark = 2L)
+    fetcher.setLeaderState(partition, leaderState)
+
+    // initial truncation and initial error response handling
+    fetcher.doWork()
+    assertEquals(Option(Fetching), fetcher.fetchState(partition).map(_.state))
+
+    TestUtils.waitUntilTrue(() => {
+      fetcher.doWork()
+      fetcher.replicaPartitionState(partition).log == fetcher.leaderPartitionState(partition).log
+    }, "Failed to reconcile leader and follower logs")
+
+    assertEquals(leaderState.logStartOffset, replicaState.logStartOffset)
+    assertEquals(leaderState.logEndOffset, replicaState.logEndOffset)
+    assertEquals(leaderState.highWatermark, replicaState.highWatermark)
+  }
+
+  @Test
   def testCorruptMessage(): Unit = {
     val partition = new TopicPartition("topic", 0)
 
     val fetcher = new MockFetcherThread {
       var fetchedOnce = false
-      override def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, PD)] = {
+      override def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, FetchData)] = {
         val fetchedData = super.fetchFromLeader(fetchRequest)
         if (!fetchedOnce) {
           val records = fetchedData.head._2.records.asInstanceOf[MemoryRecords]
@@ -245,12 +484,12 @@ class AbstractFetcherThreadTest {
       }
     }
 
-    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState())
-    fetcher.addPartitions(Map(partition -> 0L))
+    fetcher.setReplicaState(partition, MockFetcherThread.PartitionState(leaderEpoch = 0))
+    fetcher.addPartitions(Map(partition -> offsetAndEpoch(0L, leaderEpoch = 0)))
 
     val batch = mkBatch(baseOffset = 0L, leaderEpoch = 0,
       new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))
-    val leaderState = MockFetcherThread.PartitionState(Seq(batch), highWatermark = 2L)
+    val leaderState = MockFetcherThread.PartitionState(Seq(batch), leaderEpoch = 0, highWatermark = 2L)
     fetcher.setLeaderState(partition, leaderState)
 
     fetcher.doWork() // fails with corrupt record
@@ -262,19 +501,20 @@ class AbstractFetcherThreadTest {
 
   object MockFetcherThread {
     class PartitionState(var log: mutable.Buffer[RecordBatch],
+                         var leaderEpoch: Int,
                          var logStartOffset: Long,
                          var logEndOffset: Long,
                          var highWatermark: Long)
 
     object PartitionState {
-      def apply(log: Seq[RecordBatch], highWatermark: Long): PartitionState = {
+      def apply(log: Seq[RecordBatch], leaderEpoch: Int, highWatermark: Long): PartitionState = {
         val logStartOffset = log.headOption.map(_.baseOffset).getOrElse(0L)
         val logEndOffset = log.lastOption.map(_.nextOffset).getOrElse(0L)
-        new PartitionState(log.toBuffer, logStartOffset, logEndOffset, highWatermark)
+        new PartitionState(log.toBuffer, leaderEpoch, logStartOffset, logEndOffset, highWatermark)
       }
 
-      def apply(): PartitionState = {
-        apply(Seq(), 0L)
+      def apply(leaderEpoch: Int): PartitionState = {
+        apply(Seq(), leaderEpoch = leaderEpoch, highWatermark = 0L)
       }
     }
   }
@@ -309,7 +549,7 @@ class AbstractFetcherThreadTest {
 
     override def processPartitionData(topicPartition: TopicPartition,
                                       fetchOffset: Long,
-                                      partitionData: PD): Option[LogAppendInfo] = {
+                                      partitionData: FetchData): Option[LogAppendInfo] = {
       val state = replicaPartitionState(topicPartition)
 
       // Throw exception if the fetchOffset does not match the fetcherThread partition state
@@ -353,12 +593,12 @@ class AbstractFetcherThreadTest {
     }
 
     override def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = {
-     val state = replicaPartitionState(topicPartition)
-     state.log = state.log.takeWhile { batch =>
-       batch.lastOffset < truncationState.offset
-     }
-     state.logEndOffset = state.log.lastOption.map(_.lastOffset + 1).getOrElse(state.logStartOffset)
-     state.highWatermark = math.min(state.highWatermark, state.logEndOffset)
+      val state = replicaPartitionState(topicPartition)
+      state.log = state.log.takeWhile { batch =>
+        batch.lastOffset < truncationState.offset
+      }
+      state.logEndOffset = state.log.lastOption.map(_.lastOffset + 1).getOrElse(state.logStartOffset)
+      state.highWatermark = math.min(state.highWatermark, state.logEndOffset)
     }
 
     override def truncateFullyAndStartAt(topicPartition: TopicPartition, offset: Long): Unit = {
@@ -375,7 +615,7 @@ class AbstractFetcherThreadTest {
         if (state.isReadyForFetch) {
           val replicaState = replicaPartitionState(partition)
           fetchData.put(partition, new FetchRequest.PartitionData(state.fetchOffset, replicaState.logStartOffset,
-            1024 * 1024, Optional.empty()))
+            1024 * 1024, Optional.of[Integer](state.currentLeaderEpoch)))
         }
       }
       val fetchRequest = FetchRequest.Builder.forReplica(ApiKeys.FETCH.latestVersion, replicaId, 0, 1, fetchData.asJava)
@@ -390,45 +630,69 @@ class AbstractFetcherThreadTest {
     override def logEndOffset(topicPartition: TopicPartition): Long = replicaPartitionState(topicPartition).logEndOffset
 
     override def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch] = {
-      lookupEndOffsetForEpoch(epoch, replicaPartitionState(topicPartition))
+      val epochData = new EpochData(Optional.empty[Integer](), epoch)
+      val result = lookupEndOffsetForEpoch(epochData, replicaPartitionState(topicPartition))
+      if (result.endOffset == EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
+        None
+      else
+        Some(OffsetAndEpoch(result.endOffset, result.leaderEpoch))
+    }
+
+    private def checkExpectedLeaderEpoch(expectedEpochOpt: Optional[Integer],
+                                         partitionState: PartitionState): Option[Errors] = {
+      if (expectedEpochOpt.isPresent) {
+        val expectedEpoch = expectedEpochOpt.get
+        if (expectedEpoch < partitionState.leaderEpoch)
+          Some(Errors.FENCED_LEADER_EPOCH)
+        else if (expectedEpoch > partitionState.leaderEpoch)
+          Some(Errors.UNKNOWN_LEADER_EPOCH)
+        else
+          None
+      } else {
+        None
+      }
     }
 
-    private def lookupEndOffsetForEpoch(epoch: Int, partitionState: PartitionState): Option[OffsetAndEpoch] = {
+    private def lookupEndOffsetForEpoch(epochData: EpochData,
+                                        partitionState: PartitionState): EpochEndOffset = {
+      checkExpectedLeaderEpoch(epochData.currentLeaderEpoch, partitionState).foreach { error =>
+        return new EpochEndOffset(error, EpochEndOffset.UNDEFINED_EPOCH, EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
+      }
+
       var epochLowerBound = EpochEndOffset.UNDEFINED_EPOCH
       for (batch <- partitionState.log) {
-        if (batch.partitionLeaderEpoch > epoch) {
-          return Some(OffsetAndEpoch(batch.baseOffset, epochLowerBound))
+        if (batch.partitionLeaderEpoch > epochData.leaderEpoch) {
+          return new EpochEndOffset(Errors.NONE, epochLowerBound, batch.baseOffset)
         }
         epochLowerBound = batch.partitionLeaderEpoch
       }
-      None
+      new EpochEndOffset(Errors.NONE, EpochEndOffset.UNDEFINED_EPOCH, EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
     }
 
-    override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = {
+    override def fetchEpochsFromLeader(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = {
       val endOffsets = mutable.Map[TopicPartition, EpochEndOffset]()
-      partitions.foreach { case (partition, epoch) =>
-        val state = leaderPartitionState(partition)
-        val epochEndOffset = lookupEndOffsetForEpoch(epoch, state) match {
-          case Some(OffsetAndEpoch(offset, epoch)) =>
-            new EpochEndOffset(Errors.NONE, epoch, offset)
-          case None =>
-            new EpochEndOffset(Errors.NONE, EpochEndOffset.UNDEFINED_EPOCH, EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
-        }
+      partitions.foreach { case (partition, epochData) =>
+        val leaderState = leaderPartitionState(partition)
+        val epochEndOffset = lookupEndOffsetForEpoch(epochData, leaderState)
         endOffsets.put(partition, epochEndOffset)
       }
       endOffsets
     }
 
-    override def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, PD)] = {
+    override def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, FetchData)] = {
       fetchRequest.fetchData.asScala.map { case (partition, fetchData) =>
-        val state = leaderPartitionState(partition)
-        val (error, records) = if (fetchData.fetchOffset > state.logEndOffset || fetchData.fetchOffset < state.logStartOffset) {
+        val leaderState = leaderPartitionState(partition)
+        val epochCheckError = checkExpectedLeaderEpoch(fetchData.currentLeaderEpoch, leaderState)
+
+        val (error, records) = if (epochCheckError.isDefined) {
+          (epochCheckError.get, MemoryRecords.EMPTY)
+        } else if (fetchData.fetchOffset > leaderState.logEndOffset || fetchData.fetchOffset < leaderState.logStartOffset) {
           (Errors.OFFSET_OUT_OF_RANGE, MemoryRecords.EMPTY)
         } else {
           // for simplicity, we fetch only one batch at a time
-          val records = state.log.find(_.baseOffset >= fetchData.fetchOffset) match {
+          val records = leaderState.log.find(_.baseOffset >= fetchData.fetchOffset) match {
             case Some(batch) =>
-              val buffer = ByteBuffer.allocate(batch.sizeInBytes())
+              val buffer = ByteBuffer.allocate(batch.sizeInBytes)
               batch.writeTo(buffer)
               buffer.flip()
               MemoryRecords.readableRecords(buffer)
@@ -440,18 +704,29 @@ class AbstractFetcherThreadTest {
           (Errors.NONE, records)
         }
 
-        (partition, new PD(error, state.highWatermark, state.highWatermark, state.logStartOffset,
+        (partition, new FetchData(error, leaderState.highWatermark, leaderState.highWatermark, leaderState.logStartOffset,
           List.empty.asJava, records))
       }.toSeq
     }
 
-    override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition): Long = {
-      leaderPartitionState(topicPartition).logStartOffset
+    private def checkLeaderEpochAndThrow(expectedEpoch: Int, partitionState: PartitionState): Unit = {
+      checkExpectedLeaderEpoch(Optional.of[Integer](expectedEpoch), partitionState).foreach { error =>
+        throw error.exception()
+      }
+    }
+
+    override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, leaderEpoch: Int): Long = {
+      val leaderState = leaderPartitionState(topicPartition)
+      checkLeaderEpochAndThrow(leaderEpoch, leaderState)
+      leaderState.logStartOffset
     }
 
-    override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition): Long = {
-      leaderPartitionState(topicPartition).logEndOffset
+    override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, leaderEpoch: Int): Long = {
+      val leaderState = leaderPartitionState(topicPartition)
+      checkLeaderEpochAndThrow(leaderEpoch, leaderState)
+      leaderState.logEndOffset
     }
+
   }
 
 }
diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala
index e986805..28ed81d 100644
--- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala
@@ -46,7 +46,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
     val partitionDirs1 = (0 until partitionNum).map(partition => new TopicPartition(topic, partition) -> logDir1).toMap
     val alterReplicaLogDirsResponse1 = sendAlterReplicaLogDirsRequest(partitionDirs1)
 
-    // The response should show error REPLICA_NOT_AVAILABLE for all partitions
+    // The response should show error UNKNOWN_TOPIC_OR_PARTITION for all partitions
     (0 until partitionNum).foreach { partition =>
       val tp = new TopicPartition(topic, partition)
       assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, alterReplicaLogDirsResponse1.responses().get(tp))
diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
index f8c02cf..b4315d1 100644
--- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
@@ -191,6 +191,42 @@ class FetchRequestTest extends BaseRequestTest {
     assertEquals(Errors.NOT_LEADER_FOR_PARTITION, partitionData.error)
   }
 
+  @Test
+  def testCurrentEpochValidation(): Unit = {
+    val topic = "topic"
+    val topicPartition = new TopicPartition(topic, 0)
+    val partitionToLeader = TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 3, servers)
+    val firstLeaderId = partitionToLeader(topicPartition.partition)
+
+    def assertResponseErrorForEpoch(error: Errors, brokerId: Int, leaderEpoch: Optional[Integer]): Unit = {
+      val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitionMap.put(topicPartition, new FetchRequest.PartitionData(0L, 0L, 1024, leaderEpoch))
+      val fetchRequest = FetchRequest.Builder.forConsumer(0, 1, partitionMap).build()
+      val fetchResponse = sendFetchRequest(brokerId, fetchRequest)
+      val partitionData = fetchResponse.responseData.get(topicPartition)
+      assertEquals(error, partitionData.error)
+    }
+
+    // We need a leader change in order to check epoch fencing since the first epoch is 0 and
+    // -1 is treated as having no epoch at all
+    killBroker(firstLeaderId)
+
+    // Check leader error codes
+    val secondLeaderId = TestUtils.awaitLeaderChange(servers, topicPartition, firstLeaderId)
+    val secondLeaderEpoch = TestUtils.findLeaderEpoch(secondLeaderId, topicPartition, servers)
+    assertResponseErrorForEpoch(Errors.NONE, secondLeaderId, Optional.empty())
+    assertResponseErrorForEpoch(Errors.NONE, secondLeaderId, Optional.of(secondLeaderEpoch))
+    assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, secondLeaderId, Optional.of(secondLeaderEpoch - 1))
+    assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, secondLeaderId, Optional.of(secondLeaderEpoch + 1))
+
+    // Check follower error codes
+    val followerId = TestUtils.findFollowerId(topicPartition, servers)
+    assertResponseErrorForEpoch(Errors.NOT_LEADER_FOR_PARTITION, followerId, Optional.empty())
+    assertResponseErrorForEpoch(Errors.NOT_LEADER_FOR_PARTITION, followerId, Optional.of(secondLeaderEpoch))
+    assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch + 1))
+    assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1))
+  }
+
   /**
    * Tests that down-conversions dont leak memory. Large down conversions are triggered
    * in the server. The client closes its connection after reading partial data when the
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index c0871a7..7451234 100755
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -83,7 +83,7 @@ class HighwatermarkPersistenceTest {
       fooPartition0Hw = hwmFor(replicaManager, topic, 0)
       assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw)
       // set the high watermark for local replica
-      partition0.getReplica().get.highWatermark = new LogOffsetMetadata(5L)
+      partition0.localReplica.get.highWatermark = new LogOffsetMetadata(5L)
       replicaManager.checkpointHighWatermarks()
       fooPartition0Hw = hwmFor(replicaManager, topic, 0)
       assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw)
@@ -127,7 +127,7 @@ class HighwatermarkPersistenceTest {
       topic1Partition0Hw = hwmFor(replicaManager, topic1, 0)
       assertEquals(leaderReplicaTopic1Partition0.highWatermark.messageOffset, topic1Partition0Hw)
       // set the high watermark for local replica
-      topic1Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(5L)
+      topic1Partition0.localReplica.get.highWatermark = new LogOffsetMetadata(5L)
       replicaManager.checkpointHighWatermarks()
       topic1Partition0Hw = hwmFor(replicaManager, topic1, 0)
       assertEquals(5L, leaderReplicaTopic1Partition0.highWatermark.messageOffset)
@@ -144,10 +144,10 @@ class HighwatermarkPersistenceTest {
       var topic2Partition0Hw = hwmFor(replicaManager, topic2, 0)
       assertEquals(leaderReplicaTopic2Partition0.highWatermark.messageOffset, topic2Partition0Hw)
       // set the highwatermark for local replica
-      topic2Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(15L)
+      topic2Partition0.localReplica.get.highWatermark = new LogOffsetMetadata(15L)
       assertEquals(15L, leaderReplicaTopic2Partition0.highWatermark.messageOffset)
       // change the highwatermark for topic1
-      topic1Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(10L)
+      topic1Partition0.localReplica.get.highWatermark = new LogOffsetMetadata(10L)
       assertEquals(10L, leaderReplicaTopic1Partition0.highWatermark.messageOffset)
       replicaManager.checkpointHighWatermarks()
       // verify checkpointed hw for topic 2
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index eb3f28e..b903c4a 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -23,11 +23,10 @@ import java.util
 import java.util.{Collections, Optional}
 
 import kafka.api.{ApiVersion, KAFKA_0_10_2_IV0}
-import kafka.cluster.Replica
 import kafka.controller.KafkaController
 import kafka.coordinator.group.GroupCoordinator
 import kafka.coordinator.transaction.TransactionCoordinator
-import kafka.log.{Log, TimestampOffset}
+import kafka.log.TimestampOffset
 import kafka.network.RequestChannel
 import kafka.network.RequestChannel.SendResponse
 import kafka.security.auth.Authorizer
@@ -351,88 +350,51 @@ class KafkaApisTest {
   }
 
   @Test
-  def testReadUncommittedConsumerListOffsetLimitedAtHighWatermark(): Unit = {
-    testConsumerListOffsetLimit(IsolationLevel.READ_UNCOMMITTED)
+  def testLeaderReplicaIfLocalRaisesFencedLeaderEpoch(): Unit = {
+    testListOffsetFailedGetLeaderReplica(Errors.FENCED_LEADER_EPOCH)
   }
 
   @Test
-  def testReadCommittedConsumerListOffsetLimitedAtLastStableOffset(): Unit = {
-    testConsumerListOffsetLimit(IsolationLevel.READ_COMMITTED)
-  }
-
-  private def testConsumerListOffsetLimit(isolationLevel: IsolationLevel): Unit = {
-    val tp = new TopicPartition("foo", 0)
-    val timestamp: JLong = time.milliseconds()
-    val limitOffset = 15L
-
-    val replica = EasyMock.mock(classOf[Replica])
-    val log = EasyMock.mock(classOf[Log])
-    EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(tp)).andReturn(replica)
-    if (isolationLevel == IsolationLevel.READ_UNCOMMITTED)
-      EasyMock.expect(replica.highWatermark).andReturn(LogOffsetMetadata(messageOffset = limitOffset))
-    else
-      EasyMock.expect(replica.lastStableOffset).andReturn(LogOffsetMetadata(messageOffset = limitOffset))
-    EasyMock.expect(replicaManager.getLog(tp)).andReturn(Some(log))
-    EasyMock.expect(log.fetchOffsetsByTimestamp(timestamp)).andReturn(Some(TimestampOffset(timestamp = timestamp, offset = limitOffset)))
-    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(targetTimes.asJava)
-    val (listOffsetRequest, request) = buildRequest(builder)
-    createKafkaApis().handleListOffsetRequest(request)
-
-    val response = readResponse(ApiKeys.LIST_OFFSETS, listOffsetRequest, capturedResponse).asInstanceOf[ListOffsetResponse]
-    assertTrue(response.responseData.containsKey(tp))
-
-    val partitionData = response.responseData.get(tp)
-    assertEquals(Errors.NONE, partitionData.error)
-    assertEquals(ListOffsetResponse.UNKNOWN_OFFSET, partitionData.offset)
-    assertEquals(ListOffsetResponse.UNKNOWN_TIMESTAMP, partitionData.timestamp)
+  def testLeaderReplicaIfLocalRaisesUnknownLeaderEpoch(): Unit = {
+    testListOffsetFailedGetLeaderReplica(Errors.UNKNOWN_LEADER_EPOCH)
   }
 
   @Test
-  def testReadUncommittedConsumerListOffsetEarliestOffsetEqualsHighWatermark(): Unit = {
-    testConsumerListOffsetEarliestOffsetEqualsLimit(IsolationLevel.READ_UNCOMMITTED)
+  def testLeaderReplicaIfLocalRaisesNotLeaderForPartition(): Unit = {
+    testListOffsetFailedGetLeaderReplica(Errors.NOT_LEADER_FOR_PARTITION)
   }
 
   @Test
-  def testReadCommittedConsumerListOffsetEarliestOffsetEqualsLastStableOffset(): Unit = {
-    testConsumerListOffsetEarliestOffsetEqualsLimit(IsolationLevel.READ_COMMITTED)
+  def testLeaderReplicaIfLocalRaisesUnknownTopicOrPartition(): Unit = {
+    testListOffsetFailedGetLeaderReplica(Errors.UNKNOWN_TOPIC_OR_PARTITION)
   }
 
-  private def testConsumerListOffsetEarliestOffsetEqualsLimit(isolationLevel: IsolationLevel): Unit = {
+  private def testListOffsetFailedGetLeaderReplica(error: Errors): Unit = {
     val tp = new TopicPartition("foo", 0)
-    val limitOffset = 15L
-
-    val replica = EasyMock.mock(classOf[Replica])
-    val log = EasyMock.mock(classOf[Log])
-    EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(tp)).andReturn(replica)
-    if (isolationLevel == IsolationLevel.READ_UNCOMMITTED)
-      EasyMock.expect(replica.highWatermark).andReturn(LogOffsetMetadata(messageOffset = limitOffset))
-    else
-      EasyMock.expect(replica.lastStableOffset).andReturn(LogOffsetMetadata(messageOffset = limitOffset))
-    EasyMock.expect(replicaManager.getLog(tp)).andReturn(Some(log))
-    EasyMock.expect(log.fetchOffsetsByTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP))
-      .andReturn(Some(TimestampOffset(timestamp = ListOffsetResponse.UNKNOWN_TIMESTAMP, offset = limitOffset)))
+    val isolationLevel = IsolationLevel.READ_UNCOMMITTED
+    val currentLeaderEpoch = Optional.of[Integer](15)
+
+    EasyMock.expect(replicaManager.fetchOffsetForTimestamp(tp, ListOffsetRequest.EARLIEST_TIMESTAMP,
+      Some(isolationLevel), currentLeaderEpoch, fetchOnlyFromLeader = true))
+      .andThrow(error.exception)
+
     val capturedResponse = expectNoThrottling()
-    EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, replica, log)
+    EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel)
 
     val targetTimes = Map(tp -> new ListOffsetRequest.PartitionData(ListOffsetRequest.EARLIEST_TIMESTAMP,
-      Optional.empty[Integer]()))
+      currentLeaderEpoch))
     val builder = ListOffsetRequest.Builder.forConsumer(true, isolationLevel)
       .setTargetTimes(targetTimes.asJava)
     val (listOffsetRequest, request) = buildRequest(builder)
     createKafkaApis().handleListOffsetRequest(request)
 
-    val response = readResponse(ApiKeys.LIST_OFFSETS, listOffsetRequest, capturedResponse).asInstanceOf[ListOffsetResponse]
+    val response = readResponse(ApiKeys.LIST_OFFSETS, listOffsetRequest, capturedResponse)
+      .asInstanceOf[ListOffsetResponse]
     assertTrue(response.responseData.containsKey(tp))
 
     val partitionData = response.responseData.get(tp)
-    assertEquals(Errors.NONE, partitionData.error)
-    assertEquals(limitOffset, partitionData.offset)
+    assertEquals(error, partitionData.error)
+    assertEquals(ListOffsetResponse.UNKNOWN_OFFSET, partitionData.offset)
     assertEquals(ListOffsetResponse.UNKNOWN_TIMESTAMP, partitionData.timestamp)
   }
 
@@ -499,21 +461,17 @@ class KafkaApisTest {
   private def testConsumerListOffsetLatest(isolationLevel: IsolationLevel): Unit = {
     val tp = new TopicPartition("foo", 0)
     val latestOffset = 15L
+    val currentLeaderEpoch = Optional.empty[Integer]()
 
-    val replica = EasyMock.mock(classOf[Replica])
-    val log = EasyMock.mock(classOf[Log])
-    EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(tp)).andReturn(replica)
-    if (isolationLevel == IsolationLevel.READ_UNCOMMITTED)
-      EasyMock.expect(replica.highWatermark).andReturn(LogOffsetMetadata(messageOffset = latestOffset))
-    else
-      EasyMock.expect(replica.lastStableOffset).andReturn(LogOffsetMetadata(messageOffset = latestOffset))
+    EasyMock.expect(replicaManager.fetchOffsetForTimestamp(tp, ListOffsetRequest.LATEST_TIMESTAMP,
+      Some(isolationLevel), currentLeaderEpoch, fetchOnlyFromLeader = true))
+      .andReturn(TimestampOffset(ListOffsetResponse.UNKNOWN_TIMESTAMP, latestOffset))
 
     val capturedResponse = expectNoThrottling()
-
-    EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, replica, log)
+    EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel)
 
     val targetTimes = Map(tp -> new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP,
-      Optional.empty[Integer]()))
+      currentLeaderEpoch))
     val builder = ListOffsetRequest.Builder.forConsumer(true, isolationLevel)
       .setTargetTimes(targetTimes.asJava)
     val (listOffsetRequest, request) = buildRequest(builder)
diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala
index 965413e..7f9b3e4 100644
--- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala
@@ -74,6 +74,43 @@ class ListOffsetsRequestTest extends BaseRequestTest {
     assertResponseError(Errors.REPLICA_NOT_AVAILABLE, nonReplica, debugReplicaRequest)
   }
 
+  @Test
+  def testCurrentEpochValidation(): Unit = {
+    val topic = "topic"
+    val topicPartition = new TopicPartition(topic, 0)
+    val partitionToLeader = TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 3, servers)
+    val firstLeaderId = partitionToLeader(topicPartition.partition)
+
+    def assertResponseErrorForEpoch(error: Errors, brokerId: Int, currentLeaderEpoch: Optional[Integer]): Unit = {
+      val targetTimes = Map(topicPartition -> new ListOffsetRequest.PartitionData(
+        ListOffsetRequest.EARLIEST_TIMESTAMP, currentLeaderEpoch)).asJava
+      val request = ListOffsetRequest.Builder
+        .forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
+        .setTargetTimes(targetTimes)
+        .build()
+      assertResponseError(error, brokerId, request)
+    }
+
+    // We need a leader change in order to check epoch fencing since the first epoch is 0 and
+    // -1 is treated as having no epoch at all
+    killBroker(firstLeaderId)
+
+    // Check leader error codes
+    val secondLeaderId = TestUtils.awaitLeaderChange(servers, topicPartition, firstLeaderId)
+    val secondLeaderEpoch = TestUtils.findLeaderEpoch(secondLeaderId, topicPartition, servers)
+    assertResponseErrorForEpoch(Errors.NONE, secondLeaderId, Optional.empty())
+    assertResponseErrorForEpoch(Errors.NONE, secondLeaderId, Optional.of(secondLeaderEpoch))
+    assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, secondLeaderId, Optional.of(secondLeaderEpoch - 1))
+    assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, secondLeaderId, Optional.of(secondLeaderEpoch + 1))
+
+    // Check follower error codes
+    val followerId = TestUtils.findFollowerId(topicPartition, servers)
+    assertResponseErrorForEpoch(Errors.NOT_LEADER_FOR_PARTITION, followerId, Optional.empty())
+    assertResponseErrorForEpoch(Errors.NOT_LEADER_FOR_PARTITION, followerId, Optional.of(secondLeaderEpoch))
+    assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch + 1))
+    assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1))
+  }
+
   private def assertResponseError(error: Errors, brokerId: Int, request: ListOffsetRequest): Unit = {
     val response = sendRequest(brokerId, request)
     assertEquals(request.partitionTimestamps.size, response.responseData.size)
@@ -83,7 +120,8 @@ class ListOffsetsRequestTest extends BaseRequestTest {
   }
 
   private def sendRequest(leaderId: Int, request: ListOffsetRequest): ListOffsetResponse = {
-    val response = connectAndSend(request, ApiKeys.LIST_OFFSETS, destination = brokerSocketServer(leaderId))
+    val socketServer = brokerSocketServer(leaderId)
+    val response = connectAndSend(request, ApiKeys.LIST_OFFSETS, destination = socketServer)
     ListOffsetResponse.parse(response, request.version)
   }
 
diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
index 2dcbb8a..aaf6477 100644
--- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
@@ -144,7 +144,7 @@ class LogDirFailureTest extends IntegrationTestHarness {
     }, "Expected the first message", 3000L)
 
     // Make log directory of the partition on the leader broker inaccessible by replacing it with a file
-    val replica = leaderServer.replicaManager.getReplicaOrException(partition)
+    val replica = leaderServer.replicaManager.localReplicaOrException(partition)
     val logDir = replica.log.get.dir.getParentFile
     CoreUtils.swallow(Utils.delete(logDir), this)
     logDir.createNewFile()
@@ -163,7 +163,7 @@ class LogDirFailureTest extends IntegrationTestHarness {
 
     // Wait for ReplicaHighWatermarkCheckpoint to happen so that the log directory of the topic will be offline
     TestUtils.waitUntilTrue(() => !leaderServer.logManager.isLogDirOnline(logDir.getAbsolutePath), "Expected log directory offline", 3000L)
-    assertTrue(leaderServer.replicaManager.getReplica(partition).isEmpty)
+    assertTrue(leaderServer.replicaManager.localReplica(partition).isEmpty)
 
     // The second send() should fail due to either KafkaStorageException or NotLeaderForPartitionException
     try {
diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
index 45096cc..740b28e 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -80,7 +80,7 @@ class LogOffsetTest extends BaseRequestTest {
     log.maybeIncrementLogStartOffset(3)
     log.deleteOldSegments()
 
-    val offsets = server.apis.fetchOffsets(logManager, topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 15)
+    val offsets = log.legacyFetchOffsetsBefore(ListOffsetRequest.LATEST_TIMESTAMP, 15)
     assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 3L), offsets)
 
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
@@ -108,7 +108,7 @@ class LogOffsetTest extends BaseRequestTest {
       log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0)
     log.flush()
 
-    val offsets = server.apis.fetchOffsets(logManager, topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 15)
+    val offsets = log.legacyFetchOffsetsBefore(ListOffsetRequest.LATEST_TIMESTAMP, 15)
     assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets)
 
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
@@ -168,7 +168,7 @@ class LogOffsetTest extends BaseRequestTest {
 
     val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
 
-    val offsets = server.apis.fetchOffsets(logManager, topicPartition, now, 15)
+    val offsets = log.legacyFetchOffsetsBefore(now, 15)
     assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets)
 
     TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server),
@@ -194,7 +194,7 @@ class LogOffsetTest extends BaseRequestTest {
       log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0)
     log.flush()
 
-    val offsets = server.apis.fetchOffsets(logManager, topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, 10)
+    val offsets = log.legacyFetchOffsetsBefore(ListOffsetRequest.EARLIEST_TIMESTAMP, 10)
 
     assertEquals(Seq(0L), offsets)
 
@@ -221,7 +221,7 @@ class LogOffsetTest extends BaseRequestTest {
     val logSegments = Seq(logSegment)
     EasyMock.expect(log.logSegments).andStubReturn(logSegments)
     EasyMock.replay(log)
-    server.apis.fetchOffsetsBefore(log, System.currentTimeMillis, 100)
+    log.legacyFetchOffsetsBefore(System.currentTimeMillis, 100)
   }
 
   /* We test that `fetchOffsetsBefore` works correctly if `Log.logSegments` content and size are
@@ -240,7 +240,7 @@ class LogOffsetTest extends BaseRequestTest {
     }
     EasyMock.replay(logSegment)
     EasyMock.replay(log)
-    server.apis.fetchOffsetsBefore(log, System.currentTimeMillis, 100)
+    log.legacyFetchOffsetsBefore(System.currentTimeMillis, 100)
   }
 
   private def server: KafkaServer = servers.head
diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
index 82b95a8..15f9a9b 100755
--- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
@@ -104,7 +104,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
 
     // give some time for the follower 1 to record leader HW
     TestUtils.waitUntilTrue(() =>
-      server2.replicaManager.getReplica(topicPartition).get.highWatermark.messageOffset == numMessages,
+      server2.replicaManager.localReplica(topicPartition).get.highWatermark.messageOffset == numMessages,
       "Failed to update high watermark for follower after timeout")
 
     servers.foreach(_.replicaManager.checkpointHighWatermarks())
@@ -166,7 +166,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
 
     // give some time for follower 1 to record leader HW of 60
     TestUtils.waitUntilTrue(() =>
-      server2.replicaManager.getReplica(topicPartition).get.highWatermark.messageOffset == hw,
+      server2.replicaManager.localReplica(topicPartition).get.highWatermark.messageOffset == hw,
       "Failed to update high watermark for follower after timeout")
     // shutdown the servers to allow the hw to be checkpointed
     servers.foreach(_.shutdown())
@@ -180,7 +180,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
     val hw = 20L
     // give some time for follower 1 to record leader HW of 600
     TestUtils.waitUntilTrue(() =>
-      server2.replicaManager.getReplica(topicPartition).get.highWatermark.messageOffset == hw,
+      server2.replicaManager.localReplica(topicPartition).get.highWatermark.messageOffset == hw,
       "Failed to update high watermark for follower after timeout")
     // shutdown the servers to allow the hw to be checkpointed
     servers.foreach(_.shutdown())
@@ -199,7 +199,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
 
     // allow some time for the follower to get the leader HW
     TestUtils.waitUntilTrue(() =>
-      server2.replicaManager.getReplica(topicPartition).get.highWatermark.messageOffset == hw,
+      server2.replicaManager.localReplica(topicPartition).get.highWatermark.messageOffset == hw,
       "Failed to update high watermark for follower after timeout")
     // kill the server hosting the preferred replica
     server1.shutdown()
@@ -226,11 +226,11 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
     hw += 2
 
     // allow some time for the follower to create replica
-    TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topicPartition).nonEmpty,
+    TestUtils.waitUntilTrue(() => server1.replicaManager.localReplica(topicPartition).nonEmpty,
       "Failed to create replica in follower after timeout")
     // allow some time for the follower to get the leader HW
     TestUtils.waitUntilTrue(() =>
-      server1.replicaManager.getReplica(topicPartition).get.highWatermark.messageOffset == hw,
+      server1.replicaManager.localReplica(topicPartition).get.highWatermark.messageOffset == hw,
       "Failed to update high watermark for follower after timeout")
     // shutdown the servers to allow the hw to be checkpointed
     servers.foreach(_.shutdown())
diff --git a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala
index e314b44..2cdd2e8 100644
--- a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala
@@ -34,9 +34,8 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
     val topic = "topic"
     val partition = new TopicPartition(topic, 0)
 
-    val request = new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion)
-      .add(partition, Optional.of(5), 0)
-      .build()
+    val epochs = Map(partition -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty[Integer], 0)).asJava
+    val request = new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, epochs).build()
 
     // Unknown topic
     val randomBrokerId = servers.head.config.brokerId
@@ -52,6 +51,41 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
     assertResponseError(Errors.NOT_LEADER_FOR_PARTITION, nonReplica, request)
   }
 
+  @Test
+  def testCurrentEpochValidation(): Unit = {
+    val topic = "topic"
+    val topicPartition = new TopicPartition(topic, 0)
+    val partitionToLeader = TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 3, servers)
+    val firstLeaderId = partitionToLeader(topicPartition.partition)
+
+    def assertResponseErrorForEpoch(error: Errors, brokerId: Int, currentLeaderEpoch: Optional[Integer]): Unit = {
+      val epochs = Map(topicPartition -> new OffsetsForLeaderEpochRequest.PartitionData(
+        currentLeaderEpoch, 0)).asJava
+      val request = new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, epochs)
+        .build()
+      assertResponseError(error, brokerId, request)
+    }
+
+    // We need a leader change in order to check epoch fencing since the first epoch is 0 and
+    // -1 is treated as having no epoch at all
+    killBroker(firstLeaderId)
+
+    // Check leader error codes
+    val secondLeaderId = TestUtils.awaitLeaderChange(servers, topicPartition, firstLeaderId)
+    val secondLeaderEpoch = TestUtils.findLeaderEpoch(secondLeaderId, topicPartition, servers)
+    assertResponseErrorForEpoch(Errors.NONE, secondLeaderId, Optional.empty())
+    assertResponseErrorForEpoch(Errors.NONE, secondLeaderId, Optional.of(secondLeaderEpoch))
+    assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, secondLeaderId, Optional.of(secondLeaderEpoch - 1))
+    assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, secondLeaderId, Optional.of(secondLeaderEpoch + 1))
+
+    // Check follower error codes
+    val followerId = TestUtils.findFollowerId(topicPartition, servers)
+    assertResponseErrorForEpoch(Errors.NOT_LEADER_FOR_PARTITION, followerId, Optional.empty())
+    assertResponseErrorForEpoch(Errors.NOT_LEADER_FOR_PARTITION, followerId, Optional.of(secondLeaderEpoch))
+    assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch + 1))
+    assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1))
+  }
+
   private def assertResponseError(error: Errors, brokerId: Int, request: OffsetsForLeaderEpochRequest): Unit = {
     val response = sendRequest(brokerId, request)
     assertEquals(request.epochsByTopicPartition.size, response.responses.size)
@@ -60,8 +94,8 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
     }
   }
 
-  private def sendRequest(leaderId: Int, request: OffsetsForLeaderEpochRequest): OffsetsForLeaderEpochResponse = {
-    val response = connectAndSend(request, ApiKeys.OFFSET_FOR_LEADER_EPOCH, destination = brokerSocketServer(leaderId))
+  private def sendRequest(brokerId: Int, request: OffsetsForLeaderEpochRequest): OffsetsForLeaderEpochResponse = {
+    val response = connectAndSend(request, ApiKeys.OFFSET_FOR_LEADER_EPOCH, destination = brokerSocketServer(brokerId))
     OffsetsForLeaderEpochResponse.parse(response, request.version)
   }
 }
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala
index 2e28ee1..5d92b61 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala
@@ -16,51 +16,63 @@
   */
 package kafka.server
 
+import java.util.Optional
 
-import kafka.api.Request
 import kafka.cluster.{BrokerEndPoint, Partition, Replica}
 import kafka.log.LogManager
 import kafka.server.AbstractFetcherThread.ResultWithPartitions
 import kafka.server.epoch.LeaderEpochFileCache
 import kafka.utils.{DelayedItem, TestUtils}
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{KafkaStorageException, ReplicaNotAvailableException}
+import org.apache.kafka.common.errors.KafkaStorageException
 import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.EpochEndOffset
-import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH_OFFSET, UNDEFINED_EPOCH}
+import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest}
+import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
 import org.easymock.EasyMock._
 import org.easymock.{Capture, CaptureType, EasyMock, IAnswer}
 import org.junit.Assert._
 import org.junit.Test
 
 import scala.collection.JavaConverters._
-import scala.collection.{Seq, Map}
+import scala.collection.{Map, Seq}
 
 class ReplicaAlterLogDirsThreadTest {
 
   private val t1p0 = new TopicPartition("topic1", 0)
   private val t1p1 = new TopicPartition("topic1", 1)
 
+  private def offsetAndEpoch(fetchOffset: Long, leaderEpoch: Int = 1): OffsetAndEpoch = {
+    OffsetAndEpoch(offset = fetchOffset, leaderEpoch = leaderEpoch)
+  }
+
   @Test
   def issuesEpochRequestFromLocalReplica(): Unit = {
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
 
     //Setup all dependencies
-    val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache])
-    val replica = createNiceMock(classOf[Replica])
-    val futureReplica = createNiceMock(classOf[Replica])
-    val partition = createMock(classOf[Partition])
+    val partitionT1p0 = createMock(classOf[Partition])
+    val partitionT1p1 = createMock(classOf[Partition])
     val replicaManager = createMock(classOf[ReplicaManager])
 
-    val leaderEpoch = 2
-    val leo = 13
+    val leaderEpochT1p0 = 2
+    val leaderEpochT1p1 = 5
+    val leoT1p0 = 13
+    val leoT1p1 = 232
 
     //Stubs
-    expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
-    expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, leo)).anyTimes()
-    stub(replica, replica, futureReplica, partition, replicaManager)
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+      .andStubReturn(partitionT1p0)
+    expect(partitionT1p0.lastOffsetForLeaderEpoch(Optional.empty(), leaderEpochT1p0, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpochT1p0, leoT1p0))
+      .anyTimes()
+
+    expect(replicaManager.getPartitionOrException(t1p1, expectLeader = false))
+      .andStubReturn(partitionT1p1)
+    expect(partitionT1p1.lastOffsetForLeaderEpoch(Optional.empty(), leaderEpochT1p1, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpochT1p1, leoT1p1))
+      .anyTimes()
 
-    replay(leaderEpochs, replicaManager, replica)
+    replay(partitionT1p0, partitionT1p1, replicaManager)
 
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
     val thread = new ReplicaAlterLogDirsThread(
@@ -71,11 +83,13 @@ class ReplicaAlterLogDirsThreadTest {
       quota = null,
       brokerTopicStats = null)
 
-    val result = thread.fetchEpochsFromLeader(Map(t1p0 -> leaderEpoch, t1p1 -> leaderEpoch))
+    val result = thread.fetchEpochsFromLeader(Map(
+      t1p0 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), leaderEpochT1p0),
+      t1p1 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), leaderEpochT1p1)))
 
     val expected = Map(
-      t1p0 -> new EpochEndOffset(Errors.NONE, leaderEpoch, leo),
-      t1p1 -> new EpochEndOffset(Errors.NONE, leaderEpoch, leo)
+      t1p0 -> new EpochEndOffset(Errors.NONE, leaderEpochT1p0, leoT1p0),
+      t1p1 -> new EpochEndOffset(Errors.NONE, leaderEpochT1p1, leoT1p1)
     )
 
     assertEquals("results from leader epoch request should have offset from local replica",
@@ -87,23 +101,23 @@ class ReplicaAlterLogDirsThreadTest {
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
 
     //Setup all dependencies
-    val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache])
-    val replica = createNiceMock(classOf[Replica])
-    val partition = createMock(classOf[Partition])
+    val partitionT1p0 = createMock(classOf[Partition])
     val replicaManager = createMock(classOf[ReplicaManager])
 
     val leaderEpoch = 2
     val leo = 13
 
     //Stubs
-    expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
-    expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, leo)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replica).anyTimes()
-    expect(replicaManager.getPartition(t1p0)).andReturn(Some(partition)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p1)).andThrow(new KafkaStorageException).once()
-    expect(replicaManager.getPartition(t1p1)).andReturn(Some(partition)).anyTimes()
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+      .andStubReturn(partitionT1p0)
+    expect(partitionT1p0.lastOffsetForLeaderEpoch(Optional.empty(), leaderEpoch, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpoch, leo))
+      .anyTimes()
+
+    expect(replicaManager.getPartitionOrException(t1p1, expectLeader = false))
+      .andThrow(new KafkaStorageException).once()
 
-    replay(leaderEpochs, replicaManager, replica)
+    replay(partitionT1p0, replicaManager)
 
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
     val thread = new ReplicaAlterLogDirsThread(
@@ -114,7 +128,9 @@ class ReplicaAlterLogDirsThreadTest {
       quota = null,
       brokerTopicStats = null)
 
-    val result = thread.fetchEpochsFromLeader(Map(t1p0 -> leaderEpoch, t1p1 -> leaderEpoch))
+    val result = thread.fetchEpochsFromLeader(Map(
+      t1p0 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), leaderEpoch),
+      t1p1 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), leaderEpoch)))
 
     val expected = Map(
       t1p0 -> new EpochEndOffset(Errors.NONE, leaderEpoch, leo),
@@ -128,20 +144,22 @@ class ReplicaAlterLogDirsThreadTest {
   def shouldTruncateToReplicaOffset(): Unit = {
 
     //Create a capture to track what partitions/offsets are truncated
-    val truncateToCapture: Capture[Long] = newCapture(CaptureType.ALL)
+    val truncateCaptureT1p0: Capture[Long] = newCapture(CaptureType.ALL)
+    val truncateCaptureT1p1: Capture[Long] = newCapture(CaptureType.ALL)
 
     // Setup all the dependencies
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
     val quotaManager = createNiceMock(classOf[ReplicationQuotaManager])
-    val leaderEpochsT1p0 = createMock(classOf[LeaderEpochFileCache])
-    val leaderEpochsT1p1 = createMock(classOf[LeaderEpochFileCache])
-    val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache])
+    val futureReplicaLeaderEpochsT1p0 = createMock(classOf[LeaderEpochFileCache])
+    val futureReplicaLeaderEpochsT1p1 = createMock(classOf[LeaderEpochFileCache])
     val logManager = createMock(classOf[LogManager])
     val replicaT1p0 = createNiceMock(classOf[Replica])
     val replicaT1p1 = createNiceMock(classOf[Replica])
     // one future replica mock because our mocking methods return same values for both future replicas
-    val futureReplica = createNiceMock(classOf[Replica])
-    val partition = createMock(classOf[Partition])
+    val futureReplicaT1p0 = createNiceMock(classOf[Replica])
+    val futureReplicaT1p1 = createNiceMock(classOf[Replica])
+    val partitionT1p0 = createMock(classOf[Partition])
+    val partitionT1p1 = createMock(classOf[Partition])
     val replicaManager = createMock(classOf[ReplicaManager])
     val responseCallback: Capture[Seq[(TopicPartition, FetchPartitionData)] => Unit]  = EasyMock.newCapture()
 
@@ -151,21 +169,37 @@ class ReplicaAlterLogDirsThreadTest {
     val replicaT1p1LEO = 192
 
     //Stubs
-    expect(partition.truncateTo(capture(truncateToCapture), anyBoolean())).anyTimes()
-    expect(replicaT1p0.epochs).andReturn(Some(leaderEpochsT1p0)).anyTimes()
-    expect(replicaT1p1.epochs).andReturn(Some(leaderEpochsT1p1)).anyTimes()
-    expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
-    expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
-    expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch).anyTimes()
-    expect(leaderEpochsT1p0.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaT1p0LEO)).anyTimes()
-    expect(leaderEpochsT1p1.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaT1p1LEO)).anyTimes()
-    expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)).anyTimes()
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+      .andStubReturn(partitionT1p0)
+    expect(replicaManager.getPartitionOrException(t1p1, expectLeader = false))
+      .andStubReturn(partitionT1p1)
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andStubReturn(futureReplicaT1p0)
+    expect(replicaManager.futureLocalReplicaOrException(t1p1)).andStubReturn(futureReplicaT1p1)
+    expect(partitionT1p0.truncateTo(capture(truncateCaptureT1p0), anyBoolean())).anyTimes()
+    expect(partitionT1p1.truncateTo(capture(truncateCaptureT1p1), anyBoolean())).anyTimes()
+
+    expect(futureReplicaT1p0.epochs).andReturn(Some(futureReplicaLeaderEpochsT1p0)).anyTimes()
+    expect(futureReplicaT1p0.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
+    expect(futureReplicaT1p1.epochs).andReturn(Some(futureReplicaLeaderEpochsT1p1)).anyTimes()
+    expect(futureReplicaT1p1.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
+
+    expect(futureReplicaLeaderEpochsT1p0.latestEpoch).andReturn(leaderEpoch).anyTimes()
+    expect(futureReplicaLeaderEpochsT1p0.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)).anyTimes()
+    expect(partitionT1p0.lastOffsetForLeaderEpoch(Optional.of(1), leaderEpoch, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpoch, replicaT1p0LEO))
+      .anyTimes()
+
+    expect(futureReplicaLeaderEpochsT1p1.latestEpoch).andReturn(leaderEpoch).anyTimes()
+    expect(futureReplicaLeaderEpochsT1p1.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)).anyTimes()
+    expect(partitionT1p1.lastOffsetForLeaderEpoch(Optional.of(1), leaderEpoch, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpoch, replicaT1p1LEO))
+      .anyTimes()
 
     expect(replicaManager.logManager).andReturn(logManager).anyTimes()
-    stubWithFetchMessages(replicaT1p0, replicaT1p1, futureReplica, partition, replicaManager, responseCallback)
+    stubWithFetchMessages(replicaT1p0, replicaT1p1, futureReplicaT1p0, partitionT1p0, replicaManager, responseCallback)
 
-    replay(leaderEpochsT1p0, leaderEpochsT1p1, futureReplicaLeaderEpochs, replicaManager,
-           logManager, quotaManager, replicaT1p0, replicaT1p1, futureReplica, partition)
+    replay(futureReplicaLeaderEpochsT1p0, futureReplicaLeaderEpochsT1p1, replicaManager, logManager, quotaManager,
+      replicaT1p0, replicaT1p1, futureReplicaT1p0, partitionT1p0, partitionT1p1)
 
     //Create the thread
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
@@ -176,14 +210,14 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t1p1 -> offsetAndEpoch(0L)))
 
     //Run it
     thread.doWork()
 
     //We should have truncated to the offsets in the response
-    assertTrue(truncateToCapture.getValues.asScala.contains(replicaT1p0LEO))
-    assertTrue(truncateToCapture.getValues.asScala.contains(futureReplicaLEO))
+    assertEquals(replicaT1p0LEO, truncateCaptureT1p0.getValue)
+    assertEquals(futureReplicaLEO, truncateCaptureT1p1.getValue)
   }
 
   @Test
@@ -195,7 +229,6 @@ class ReplicaAlterLogDirsThreadTest {
     // Setup all the dependencies
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
     val quotaManager = createNiceMock(classOf[ReplicationQuotaManager])
-    val leaderEpochs = createMock(classOf[LeaderEpochFileCache])
     val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache])
     val logManager = createMock(classOf[LogManager])
     val replica = createNiceMock(classOf[Replica])
@@ -212,25 +245,32 @@ class ReplicaAlterLogDirsThreadTest {
     val futureReplicaEpochEndOffset = 191
 
     //Stubs
-    expect(partition.truncateTo(capture(truncateToCapture), anyBoolean())).anyTimes()
-    expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+      .andStubReturn(partition)
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andStubReturn(futureReplica)
+
+    expect(partition.truncateTo(capture(truncateToCapture), EasyMock.eq(true))).anyTimes()
     expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
     expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
     expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch).once()
     expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch - 2).once()
 
     // leader replica truncated and fetched new offsets with new leader epoch
-    expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch - 1, replicaLEO)).anyTimes()
+    expect(partition.lastOffsetForLeaderEpoch(Optional.of(1), leaderEpoch, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpoch - 1, replicaLEO))
+      .anyTimes()
     // but future replica does not know about this leader epoch, so returns a smaller leader epoch
     expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch - 1)).andReturn((leaderEpoch - 2, futureReplicaLEO)).anyTimes()
     // finally, the leader replica knows about the leader epoch and returns end offset
-    expect(leaderEpochs.endOffsetFor(leaderEpoch - 2)).andReturn((leaderEpoch - 2, replicaEpochEndOffset)).anyTimes()
+    expect(partition.lastOffsetForLeaderEpoch(Optional.of(1), leaderEpoch - 2, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(leaderEpoch - 2, replicaEpochEndOffset))
+      .anyTimes()
     expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch - 2)).andReturn((leaderEpoch - 2, futureReplicaEpochEndOffset)).anyTimes()
 
     expect(replicaManager.logManager).andReturn(logManager).anyTimes()
     stubWithFetchMessages(replica, replica, futureReplica, partition, replicaManager, responseCallback)
 
-    replay(leaderEpochs, futureReplicaLeaderEpochs, replicaManager, logManager, quotaManager, replica, futureReplica, partition)
+    replay(futureReplicaLeaderEpochs, replicaManager, logManager, quotaManager, replica, futureReplica, partition)
 
     //Create the thread
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
@@ -241,7 +281,7 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L)))
 
     // First run will result in another offset for leader epoch request
     thread.doWork()
@@ -265,7 +305,6 @@ class ReplicaAlterLogDirsThreadTest {
     val logManager = createMock(classOf[LogManager])
     val replica = createNiceMock(classOf[Replica])
     val futureReplica = createNiceMock(classOf[Replica])
-    val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache])
     val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache])
     val partition = createMock(classOf[Partition])
     val replicaManager = createMock(classOf[ReplicaManager])
@@ -275,21 +314,20 @@ class ReplicaAlterLogDirsThreadTest {
     val futureReplicaLEO = 111
 
     //Stubs
-    expect(partition.truncateTo(capture(truncated), anyBoolean())).anyTimes()
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+      .andStubReturn(partition)
+    expect(partition.truncateTo(capture(truncated), isFuture = EasyMock.eq(true))).anyTimes()
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andStubReturn(futureReplica)
+
     expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
     expect(replicaManager.logManager).andReturn(logManager).anyTimes()
-    expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
     expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
 
     // pretend this is a completely new future replica, with no leader epochs recorded
     expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(UNDEFINED_EPOCH).anyTimes()
 
-    // since UNDEFINED_EPOCH is -1 which will be lower than any valid leader epoch, the method
-    // will return UNDEFINED_EPOCH_OFFSET if requested epoch is lower than the first epoch cached
-    expect(leaderEpochs.endOffsetFor(UNDEFINED_EPOCH)).andReturn((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)).anyTimes()
     stubWithFetchMessages(replica, replica, futureReplica, partition, replicaManager, responseCallback)
-    replay(replicaManager, logManager, quotaManager, leaderEpochs, futureReplicaLeaderEpochs,
-           replica, futureReplica, partition)
+    replay(replicaManager, logManager, quotaManager, futureReplicaLeaderEpochs, replica, futureReplica, partition)
 
     //Create the thread
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
@@ -300,7 +338,7 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> initialFetchOffset))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(initialFetchOffset)))
 
     //Run it
     thread.doWork()
@@ -319,7 +357,6 @@ class ReplicaAlterLogDirsThreadTest {
     // Setup all the dependencies
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
     val quotaManager = createNiceMock(classOf[kafka.server.ReplicationQuotaManager])
-    val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache])
     val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache])
     val logManager = createMock(classOf[kafka.log.LogManager])
     val replica = createNiceMock(classOf[Replica])
@@ -333,22 +370,25 @@ class ReplicaAlterLogDirsThreadTest {
     val replicaLEO = 300
 
     //Stubs
-    expect(partition.truncateTo(capture(truncated), anyBoolean())).anyTimes()
-    expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
-    expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
-
-    expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(futureReplicaLeaderEpoch).anyTimes()
-    expect(leaderEpochs.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, replicaLEO)).anyTimes()
-    expect(futureReplicaLeaderEpochs.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, futureReplicaLEO)).anyTimes()
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+      .andStubReturn(partition)
+    expect(partition.truncateTo(capture(truncated), isFuture = EasyMock.eq(true))).once()
 
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andStubReturn(futureReplica)
+    expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
+    expect(futureReplicaLeaderEpochs.latestEpoch).andStubReturn(futureReplicaLeaderEpoch)
+    expect(futureReplicaLeaderEpochs.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, futureReplicaLEO))
     expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
-    expect(replicaManager.getReplica(t1p0)).andReturn(Some(replica)).anyTimes()
-    expect(replicaManager.getReplica(t1p0, Request.FutureLocalReplicaId)).andReturn(Some(futureReplica)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p0, Request.FutureLocalReplicaId)).andReturn(futureReplica).anyTimes()
+    expect(replicaManager.localReplica(t1p0)).andReturn(Some(replica)).anyTimes()
+    expect(replicaManager.futureLocalReplica(t1p0)).andReturn(Some(futureReplica)).anyTimes()
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andReturn(futureReplica).anyTimes()
+
     // this will cause fetchEpochsFromLeader return an error with undefined offset
-    expect(replicaManager.getReplicaOrException(t1p0)).andThrow(new ReplicaNotAvailableException("")).times(3)
-    expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replica).once()
-    expect(replicaManager.getPartition(t1p0)).andReturn(Some(partition)).anyTimes()
+    expect(partition.lastOffsetForLeaderEpoch(Optional.of(1), futureReplicaLeaderEpoch, fetchOnlyFromLeader = false))
+      .andReturn(new EpochEndOffset(Errors.REPLICA_NOT_AVAILABLE, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET))
+      .times(3)
+      .andReturn(new EpochEndOffset(futureReplicaLeaderEpoch, replicaLEO))
+
     expect(replicaManager.logManager).andReturn(logManager).anyTimes()
     expect(replicaManager.fetchMessages(
       EasyMock.anyLong(),
@@ -366,8 +406,7 @@ class ReplicaAlterLogDirsThreadTest {
         }
       }).anyTimes()
 
-    replay(leaderEpochs, futureReplicaLeaderEpochs, replicaManager, logManager, quotaManager,
-           replica, futureReplica, partition)
+    replay(futureReplicaLeaderEpochs, replicaManager, logManager, quotaManager, replica, futureReplica, partition)
 
     //Create the thread
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
@@ -378,12 +417,12 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L)))
 
     // Run thread 3 times (exactly number of times we mock exception for getReplicaOrException)
     (0 to 2).foreach { _ =>
       thread.doWork()
-                     }
+    }
 
     // Nothing happened since the replica was not available
     assertEquals(0, truncated.getValues.size())
@@ -401,7 +440,6 @@ class ReplicaAlterLogDirsThreadTest {
     //Setup all dependencies
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
     val quotaManager = createNiceMock(classOf[ReplicationQuotaManager])
-    val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache])
     val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache])
     val logManager = createMock(classOf[LogManager])
     val replica = createNiceMock(classOf[Replica])
@@ -414,19 +452,21 @@ class ReplicaAlterLogDirsThreadTest {
     val futureReplicaLEO = 190
     val replicaLEO = 213
 
-    //Stubs
-    expect(partition.truncateTo(futureReplicaLEO, true)).once()
-    expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
-    expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
+    expect(replicaManager.getPartitionOrException(t1p0, expectLeader = false))
+        .andStubReturn(partition)
+    expect(partition.lastOffsetForLeaderEpoch(Optional.of(1), leaderEpoch, fetchOnlyFromLeader = false))
+        .andReturn(new EpochEndOffset(leaderEpoch, replicaLEO))
+    expect(partition.truncateTo(futureReplicaLEO, isFuture = true)).once()
 
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andStubReturn(futureReplica)
+    expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes()
+    expect(futureReplicaLeaderEpochs.latestEpoch).andStubReturn(leaderEpoch)
     expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes()
-    expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch)
-    expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaLEO))
     expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO))
     expect(replicaManager.logManager).andReturn(logManager).anyTimes()
     stubWithFetchMessages(replica, replica, futureReplica, partition, replicaManager, responseCallback)
 
-    replay(leaderEpochs, futureReplicaLeaderEpochs, replicaManager, logManager, quotaManager,
+    replay(futureReplicaLeaderEpochs, replicaManager, logManager, quotaManager,
            replica, futureReplica, partition)
 
     //Create the fetcher thread
@@ -438,12 +478,12 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L)))
 
     // loop few times
     (0 to 3).foreach { _ =>
       thread.doWork()
-                     }
+    }
 
     //Assert that truncate to is called exactly once (despite more loops)
     verify(partition)
@@ -470,6 +510,7 @@ class ReplicaAlterLogDirsThreadTest {
 
     //Create the fetcher thread
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
+    val leaderEpoch = 1
     val thread = new ReplicaAlterLogDirsThread(
       "alter-logs-dirs-thread-test1",
       sourceBroker = endPoint,
@@ -477,11 +518,13 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(
+      t1p0 -> offsetAndEpoch(0L, leaderEpoch),
+      t1p1 -> offsetAndEpoch(0L, leaderEpoch)))
 
     val ResultWithPartitions(fetchRequestOpt, partitionsWithError) = thread.buildFetch(Map(
-      t1p0 -> new PartitionFetchState(150),
-      t1p1 -> new PartitionFetchState(160)))
+      t1p0 -> PartitionFetchState(150, leaderEpoch, state = Fetching),
+      t1p1 -> PartitionFetchState(160, leaderEpoch, state = Fetching)))
 
     assertTrue(fetchRequestOpt.isDefined)
     val fetchRequest = fetchRequestOpt.get
@@ -516,6 +559,7 @@ class ReplicaAlterLogDirsThreadTest {
 
     //Create the fetcher thread
     val endPoint = new BrokerEndPoint(0, "localhost", 1000)
+    val leaderEpoch = 1
     val thread = new ReplicaAlterLogDirsThread(
       "alter-logs-dirs-thread-test1",
       sourceBroker = endPoint,
@@ -523,12 +567,14 @@ class ReplicaAlterLogDirsThreadTest {
       replicaMgr = replicaManager,
       quota = quotaManager,
       brokerTopicStats = null)
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(
+      t1p0 -> offsetAndEpoch(0L, leaderEpoch),
+      t1p1 -> offsetAndEpoch(0L, leaderEpoch)))
 
     // one partition is ready and one is truncating
     val ResultWithPartitions(fetchRequestOpt, partitionsWithError) = thread.buildFetch(Map(
-        t1p0 -> new PartitionFetchState(150),
-        t1p1 -> new PartitionFetchState(160, truncatingLog=true)))
+        t1p0 -> PartitionFetchState(150, leaderEpoch, state = Fetching),
+        t1p1 -> PartitionFetchState(160, leaderEpoch, state = Truncating)))
 
     assertTrue(fetchRequestOpt.isDefined)
     val fetchRequest = fetchRequestOpt.get
@@ -541,8 +587,8 @@ class ReplicaAlterLogDirsThreadTest {
 
     // one partition is ready and one is delayed
     val ResultWithPartitions(fetchRequest2Opt, partitionsWithError2) = thread.buildFetch(Map(
-        t1p0 -> new PartitionFetchState(140),
-        t1p1 -> new PartitionFetchState(160, delay=new DelayedItem(5000))))
+        t1p0 -> PartitionFetchState(140, leaderEpoch, state = Fetching),
+        t1p1 -> PartitionFetchState(160, leaderEpoch, delay = new DelayedItem(5000), state = Fetching)))
 
     assertTrue(fetchRequest2Opt.isDefined)
     val fetchRequest2 = fetchRequest2Opt.get
@@ -555,22 +601,22 @@ class ReplicaAlterLogDirsThreadTest {
 
     // both partitions are delayed
     val ResultWithPartitions(fetchRequest3Opt, partitionsWithError3) = thread.buildFetch(Map(
-        t1p0 -> new PartitionFetchState(140, delay=new DelayedItem(5000)),
-        t1p1 -> new PartitionFetchState(160, delay=new DelayedItem(5000))))
+        t1p0 -> PartitionFetchState(140, leaderEpoch, delay = new DelayedItem(5000), state = Fetching),
+        t1p1 -> PartitionFetchState(160, leaderEpoch, delay = new DelayedItem(5000), state = Fetching)))
     assertTrue("Expected no fetch requests since all partitions are delayed", fetchRequest3Opt.isEmpty)
     assertFalse(partitionsWithError3.nonEmpty)
   }
 
   def stub(replicaT1p0: Replica, replicaT1p1: Replica, futureReplica: Replica, partition: Partition, replicaManager: ReplicaManager) = {
-    expect(replicaManager.getReplica(t1p0)).andReturn(Some(replicaT1p0)).anyTimes()
-    expect(replicaManager.getReplica(t1p0, Request.FutureLocalReplicaId)).andReturn(Some(futureReplica)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replicaT1p0).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p0, Request.FutureLocalReplicaId)).andReturn(futureReplica).anyTimes()
+    expect(replicaManager.localReplica(t1p0)).andReturn(Some(replicaT1p0)).anyTimes()
+    expect(replicaManager.futureLocalReplica(t1p0)).andReturn(Some(futureReplica)).anyTimes()
+    expect(replicaManager.localReplicaOrException(t1p0)).andReturn(replicaT1p0).anyTimes()
+    expect(replicaManager.futureLocalReplicaOrException(t1p0)).andReturn(futureReplica).anyTimes()
     expect(replicaManager.getPartition(t1p0)).andReturn(Some(partition)).anyTimes()
-    expect(replicaManager.getReplica(t1p1)).andReturn(Some(replicaT1p1)).anyTimes()
-    expect(replicaManager.getReplica(t1p1, Request.FutureLocalReplicaId)).andReturn(Some(futureReplica)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p1)).andReturn(replicaT1p1).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p1, Request.FutureLocalReplicaId)).andReturn(futureReplica).anyTimes()
+    expect(replicaManager.localReplica(t1p1)).andReturn(Some(replicaT1p1)).anyTimes()
+    expect(replicaManager.futureLocalReplica(t1p1)).andReturn(Some(futureReplica)).anyTimes()
+    expect(replicaManager.localReplicaOrException(t1p1)).andReturn(replicaT1p1).anyTimes()
+    expect(replicaManager.futureLocalReplicaOrException(t1p1)).andReturn(futureReplica).anyTimes()
     expect(replicaManager.getPartition(t1p1)).andReturn(Some(partition)).anyTimes()
   }
 
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
index 9440c29..4d54c81 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
@@ -16,6 +16,8 @@
   */
 package kafka.server
 
+import java.util.Optional
+
 import kafka.cluster.{BrokerEndPoint, Replica}
 import kafka.log.LogManager
 import kafka.cluster.Partition
@@ -29,7 +31,7 @@ import org.apache.kafka.common.internals.PartitionStates
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.protocol.Errors._
-import org.apache.kafka.common.requests.EpochEndOffset
+import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest}
 import org.apache.kafka.common.requests.EpochEndOffset._
 import org.apache.kafka.common.utils.SystemTime
 import org.easymock.EasyMock._
@@ -49,6 +51,10 @@ class ReplicaFetcherThreadTest {
   private var toFail = false
   private val brokerEndPoint = new BrokerEndPoint(0, "localhost", 1000)
 
+  private def offsetAndEpoch(fetchOffset: Long, leaderEpoch: Int = 1): OffsetAndEpoch = {
+    OffsetAndEpoch(offset = fetchOffset, leaderEpoch = leaderEpoch)
+  }
+
   @Test
   def shouldSendLatestRequestVersionsByDefault(): Unit = {
     val props = TestUtils.createBrokerConfig(1, "localhost:1234")
@@ -93,7 +99,9 @@ class ReplicaFetcherThreadTest {
       quota = null,
       leaderEndpointBlockingSend = None)
 
-    val result = thread.fetchEpochsFromLeader(Map(t1p0 -> 0, t1p1 -> 0))
+    val result = thread.fetchEpochsFromLeader(Map(
+      t1p0 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), 0),
+      t1p1 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), 0)))
 
     val expected = Map(
       t1p0 -> new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET),
@@ -137,7 +145,9 @@ class ReplicaFetcherThreadTest {
       leaderEndpointBlockingSend = Some(leaderEndpoint))
 
 
-    val result = thread.fetchEpochsFromLeader(Map(t1p0 -> UNDEFINED_EPOCH, t1p1 -> UNDEFINED_EPOCH))
+    val result = thread.fetchEpochsFromLeader(Map(
+      t1p0 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), UNDEFINED_EPOCH),
+      t1p1 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), UNDEFINED_EPOCH)))
 
     val expected = Map(
       t1p0 -> new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET),
@@ -191,29 +201,32 @@ class ReplicaFetcherThreadTest {
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
 
     // topic 1 supports epoch, t2 doesn't
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0, t2p1 -> 0))
+    thread.addPartitions(Map(
+      t1p0 -> offsetAndEpoch(0L),
+      t1p1 -> offsetAndEpoch(0L),
+      t2p1 -> offsetAndEpoch(0L)))
 
-    assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = false, shouldBeTruncatingLog = true, shouldBeDelayed = false)
+    assertPartitionStates(thread, shouldBeReadyForFetch = false, shouldBeTruncatingLog = true, shouldBeDelayed = false)
     //Loop 1
     thread.doWork()
     assertEquals(1, mockNetwork.epochFetchCount)
     assertEquals(1, mockNetwork.fetchCount)
 
-    assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
+    assertPartitionStates(thread, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
 
     //Loop 2 we should not fetch epochs
     thread.doWork()
     assertEquals(1, mockNetwork.epochFetchCount)
     assertEquals(2, mockNetwork.fetchCount)
 
-    assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
+    assertPartitionStates(thread, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
 
     //Loop 3 we should not fetch epochs
     thread.doWork()
     assertEquals(1, mockNetwork.epochFetchCount)
     assertEquals(3, mockNetwork.fetchCount)
 
-    assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
+    assertPartitionStates(thread, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
 
     //Assert that truncate to is called exactly once (despite two loops)
     verify(logManager)
@@ -223,22 +236,25 @@ class ReplicaFetcherThreadTest {
     * Assert that all partitions' states are as expected
     *
     */
-  def assertPartitionStates(states: PartitionStates[PartitionFetchState], shouldBeReadyForFetch: Boolean,
-                            shouldBeTruncatingLog: Boolean, shouldBeDelayed: Boolean): Unit = {
+  def assertPartitionStates(fetcher: AbstractFetcherThread,
+                            shouldBeReadyForFetch: Boolean,
+                            shouldBeTruncatingLog: Boolean,
+                            shouldBeDelayed: Boolean): Unit = {
     for (tp <- List(t1p0, t1p1, t2p1)) {
+      assertTrue(fetcher.fetchState(tp).isDefined)
+      val fetchState = fetcher.fetchState(tp).get
+
       assertEquals(
         s"Partition $tp should${if (!shouldBeReadyForFetch) " NOT" else ""} be ready for fetching",
-        shouldBeReadyForFetch, states.stateValue(tp).isReadyForFetch)
+        shouldBeReadyForFetch, fetchState.isReadyForFetch)
 
       assertEquals(
         s"Partition $tp should${if (!shouldBeTruncatingLog) " NOT" else ""} be truncating its log",
-        shouldBeTruncatingLog,
-        states.stateValue(tp).isTruncatingLog)
+        shouldBeTruncatingLog, fetchState.isTruncating)
 
       assertEquals(
         s"Partition $tp should${if (!shouldBeDelayed) " NOT" else ""} be delayed",
-        shouldBeDelayed,
-        states.stateValue(tp).isDelayed)
+        shouldBeDelayed, fetchState.isDelayed)
     }
   }
 
@@ -262,7 +278,9 @@ class ReplicaFetcherThreadTest {
       quota = null,
       leaderEndpointBlockingSend = Some(mockBlockingSend))
 
-    val result = thread.fetchEpochsFromLeader(Map(t1p0 -> 0, t1p1 -> 0))
+    val result = thread.fetchEpochsFromLeader(Map(
+      t1p0 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), 0),
+      t1p1 -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.empty(), 0)))
 
     val expected = Map(
       t1p0 -> new EpochEndOffset(Errors.UNKNOWN_SERVER_ERROR, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET),
@@ -309,7 +327,7 @@ class ReplicaFetcherThreadTest {
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsets, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager,
       new Metrics, new SystemTime, UnboundedQuota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t1p1 -> offsetAndEpoch(0L)))
 
     //Loop 1
     thread.doWork()
@@ -368,7 +386,7 @@ class ReplicaFetcherThreadTest {
     //Create the thread
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> 0, t2p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t2p1 -> offsetAndEpoch(0L)))
 
     //Run it
     thread.doWork()
@@ -419,7 +437,7 @@ class ReplicaFetcherThreadTest {
     //Create the thread
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> 0, t2p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t2p1 -> offsetAndEpoch(0L)))
 
     //Run it
     thread.doWork()
@@ -471,7 +489,7 @@ class ReplicaFetcherThreadTest {
     // Create the fetcher thread
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsets, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t1p1 -> offsetAndEpoch(0L)))
 
     // Loop 1 -- both topic partitions will need to fetch another leader epoch
     thread.doWork()
@@ -543,7 +561,7 @@ class ReplicaFetcherThreadTest {
     // Create the fetcher thread
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsets, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t1p1 -> offsetAndEpoch(0L)))
 
     // Loop 1 -- both topic partitions will truncate to leader offset even though they don't know
     // about leader epoch
@@ -601,7 +619,7 @@ class ReplicaFetcherThreadTest {
     //Create the thread
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> initialFetchOffset))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(initialFetchOffset)))
 
     //Run it
     thread.doWork()
@@ -652,7 +670,7 @@ class ReplicaFetcherThreadTest {
     //Create the thread
     val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, brokerEndPoint, new SystemTime())
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
-    thread.addPartitions(Map(t1p0 -> 0, t2p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t2p1 -> offsetAndEpoch(0L)))
 
     //Run thread 3 times
     (0 to 3).foreach { _ =>
@@ -708,16 +726,18 @@ class ReplicaFetcherThreadTest {
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
 
     //When
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t1p1 -> offsetAndEpoch(0L)))
 
     //Then all partitions should start in an TruncatingLog state
-    assertTrue(thread.partitionStates.partitionStates().asScala.forall(_.value().truncatingLog))
+    assertEquals(Option(Truncating), thread.fetchState(t1p0).map(_.state))
+    assertEquals(Option(Truncating), thread.fetchState(t1p1).map(_.state))
 
     //When
     thread.doWork()
 
     //Then none should be TruncatingLog anymore
-    assertFalse(thread.partitionStates.partitionStates().asScala.forall(_.value().truncatingLog))
+    assertEquals(Option(Fetching), thread.fetchState(t1p0).map(_.state))
+    assertEquals(Option(Fetching), thread.fetchState(t1p1).map(_.state))
   }
 
   @Test
@@ -758,7 +778,7 @@ class ReplicaFetcherThreadTest {
     val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
 
     //When
-    thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
+    thread.addPartitions(Map(t1p0 -> offsetAndEpoch(0L), t1p1 -> offsetAndEpoch(0L)))
 
     //When the epoch request is outstanding, remove one of the partitions to simulate a leader change. We do this via a callback passed to the mock thread
     val partitionThatBecameLeader = t1p0
@@ -774,14 +794,14 @@ class ReplicaFetcherThreadTest {
   }
 
   def stub(replica: Replica, partition: Partition, replicaManager: ReplicaManager) = {
-    expect(replicaManager.getReplica(t1p0)).andReturn(Some(replica)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replica).anyTimes()
+    expect(replicaManager.localReplica(t1p0)).andReturn(Some(replica)).anyTimes()
+    expect(replicaManager.localReplicaOrException(t1p0)).andReturn(replica).anyTimes()
     expect(replicaManager.getPartition(t1p0)).andReturn(Some(partition)).anyTimes()
-    expect(replicaManager.getReplica(t1p1)).andReturn(Some(replica)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t1p1)).andReturn(replica).anyTimes()
+    expect(replicaManager.localReplica(t1p1)).andReturn(Some(replica)).anyTimes()
+    expect(replicaManager.localReplicaOrException(t1p1)).andReturn(replica).anyTimes()
     expect(replicaManager.getPartition(t1p1)).andReturn(Some(partition)).anyTimes()
-    expect(replicaManager.getReplica(t2p1)).andReturn(Some(replica)).anyTimes()
-    expect(replicaManager.getReplicaOrException(t2p1)).andReturn(replica).anyTimes()
+    expect(replicaManager.localReplica(t2p1)).andReturn(Some(replica)).anyTimes()
+    expect(replicaManager.localReplicaOrException(t2p1)).andReturn(replica).anyTimes()
     expect(replicaManager.getPartition(t2p1)).andReturn(Some(partition)).anyTimes()
   }
 }
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index ea0a8ef..9b59e71 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -20,8 +20,8 @@ import java.io.File
 import java.util.{Optional, Properties}
 import java.util.concurrent.atomic.AtomicBoolean
 
-import kafka.cluster.Replica
-import kafka.log.Log
+import kafka.cluster.{Partition, Replica}
+import kafka.log.{Log, LogOffsetSnapshot}
 import kafka.utils._
 import kafka.zk.KafkaZkClient
 import org.apache.kafka.common.TopicPartition
@@ -61,12 +61,11 @@ class ReplicaManagerQuotasTest {
     val fetch = replicaManager.readFromLocalLog(
       replicaId = followerReplicaId,
       fetchOnlyFromLeader = true,
-      readOnlyCommitted = true,
+      fetchIsolation = FetchHighWatermark,
       fetchMaxBytes = Int.MaxValue,
       hardMaxBytesLimit = false,
       readPartitionInfo = fetchInfo,
-      quota = quota,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+      quota = quota)
     assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
       fetch.find(_._1 == topicPartition1).get._2.info.records.batches.asScala.size)
 
@@ -87,12 +86,11 @@ class ReplicaManagerQuotasTest {
     val fetch = replicaManager.readFromLocalLog(
       replicaId = followerReplicaId,
       fetchOnlyFromLeader = true,
-      readOnlyCommitted = true,
+      fetchIsolation = FetchHighWatermark,
       fetchMaxBytes = Int.MaxValue,
       hardMaxBytesLimit = false,
       readPartitionInfo = fetchInfo,
-      quota = quota,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+      quota = quota)
     assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
       fetch.find(_._1 == topicPartition1).get._2.info.records.batches.asScala.size)
     assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
@@ -112,12 +110,11 @@ class ReplicaManagerQuotasTest {
     val fetch = replicaManager.readFromLocalLog(
       replicaId = followerReplicaId,
       fetchOnlyFromLeader = true,
-      readOnlyCommitted = true,
+      fetchIsolation = FetchHighWatermark,
       fetchMaxBytes = Int.MaxValue,
       hardMaxBytesLimit = false,
       readPartitionInfo = fetchInfo,
-      quota = quota,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+      quota = quota)
     assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
       fetch.find(_._1 == topicPartition1).get._2.info.records.batches.asScala.size)
     assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
@@ -137,12 +134,11 @@ class ReplicaManagerQuotasTest {
     val fetch = replicaManager.readFromLocalLog(
       replicaId = followerReplicaId,
       fetchOnlyFromLeader = true,
-      readOnlyCommitted = true,
+      fetchIsolation = FetchHighWatermark,
       fetchMaxBytes = Int.MaxValue,
       hardMaxBytesLimit = false,
       readPartitionInfo = fetchInfo,
-      quota = quota,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+      quota = quota)
     assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
       fetch.find(_._1 == topicPartition1).get._2.info.records.batches.asScala.size)
 
@@ -154,27 +150,40 @@ class ReplicaManagerQuotasTest {
   def testCompleteInDelayedFetchWithReplicaThrottling(): Unit = {
     // Set up DelayedFetch where there is data to return to a follower replica, either in-sync or out of sync
     def setupDelayedFetch(isReplicaInSync: Boolean): DelayedFetch = {
-      val logOffsetMetadata = new LogOffsetMetadata(messageOffset = 100L, segmentBaseOffset = 0L, relativePositionInSegment = 500)
-      val replica = EasyMock.createMock(classOf[Replica])
-      EasyMock.expect(replica.logEndOffset).andReturn(logOffsetMetadata).anyTimes()
-      EasyMock.replay(replica)
+      val endOffsetMetadata = new LogOffsetMetadata(messageOffset = 100L, segmentBaseOffset = 0L, relativePositionInSegment = 500)
+      val partition = EasyMock.createMock(classOf[Partition])
+
+      val offsetSnapshot = LogOffsetSnapshot(
+        logStartOffset = 0L,
+        logEndOffset = endOffsetMetadata,
+        highWatermark = endOffsetMetadata,
+        lastStableOffset = endOffsetMetadata)
+      EasyMock.expect(partition.fetchOffsetSnapshot(Optional.empty(), fetchOnlyFromLeader = true))
+          .andReturn(offsetSnapshot)
 
       val replicaManager = EasyMock.createMock(classOf[ReplicaManager])
-      EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(EasyMock.anyObject[TopicPartition])).andReturn(replica).anyTimes()
+      EasyMock.expect(replicaManager.getPartitionOrException(
+        EasyMock.anyObject[TopicPartition], EasyMock.anyBoolean()))
+        .andReturn(partition).anyTimes()
+
       EasyMock.expect(replicaManager.shouldLeaderThrottle(EasyMock.anyObject[ReplicaQuota], EasyMock.anyObject[TopicPartition], EasyMock.anyObject[Int]))
         .andReturn(!isReplicaInSync).anyTimes()
-      EasyMock.replay(replicaManager)
+      EasyMock.replay(replicaManager, partition)
 
       val tp = new TopicPartition("t1", 0)
-      val fetchParititonStatus = FetchPartitionStatus(new LogOffsetMetadata(messageOffset = 50L, segmentBaseOffset = 0L,
+      val fetchPartitionStatus = 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)))
+      val fetchMetadata = FetchMetadata(fetchMinBytes = 1,
+        fetchMaxBytes = 1000,
+        hardMaxBytesLimit = true,
+        fetchOnlyLeader = true,
+        fetchIsolation = FetchLogEnd,
+        isFromFollower = true,
+        replicaId = 1,
+        fetchPartitionStatus = List((tp, fetchPartitionStatus)))
       new DelayedFetch(delayMs = 600, fetchMetadata = fetchMetadata, replicaManager = replicaManager,
-        quota = null, isolationLevel = IsolationLevel.READ_UNCOMMITTED, responseCallback = null) {
-        override def forceComplete(): Boolean = {
-          true
-        }
+        quota = null, responseCallback = null) {
+        override def forceComplete(): Boolean = true
       }
     }
 
@@ -193,16 +202,22 @@ class ReplicaManagerQuotasTest {
     expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(20L)).anyTimes()
 
     //if we ask for len 1 return a message
-    expect(log.read(anyObject(), geq(1), anyObject(), anyObject(),
-      EasyMock.eq(IsolationLevel.READ_UNCOMMITTED))).andReturn(
+    expect(log.read(anyObject(),
+      maxLength = geq(1),
+      maxOffset = anyObject(),
+      minOneMessage = anyBoolean(),
+      includeAbortedTxns = EasyMock.eq(false))).andReturn(
       FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
         MemoryRecords.withRecords(CompressionType.NONE, record)
       )).anyTimes()
 
     //if we ask for len = 0, return 0 messages
-    expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject(),
-      EasyMock.eq(IsolationLevel.READ_UNCOMMITTED))).andReturn(
+    expect(log.read(anyObject(),
+      maxLength = EasyMock.eq(0),
+      maxOffset = anyObject(),
+      minOneMessage = anyBoolean(),
+      includeAbortedTxns = EasyMock.eq(false))).andReturn(
       FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
         MemoryRecords.EMPTY
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 90d488d..4401748 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -166,7 +166,8 @@ class ReplicaManagerTest {
           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))
+      rm.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
+          .localReplicaOrException
 
       val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("first message".getBytes()))
       val appendResult = appendRecords(rm, new TopicPartition(topic, 0), records).onFire { response =>
@@ -210,7 +211,8 @@ class ReplicaManagerTest {
           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))
+      replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
+        .localReplicaOrException
 
       val producerId = 234L
       val epoch = 5.toShort
@@ -260,8 +262,8 @@ class ReplicaManagerTest {
           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))
-
+      replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
+        .localReplicaOrException
 
       val producerId = 234L
       val epoch = 5.toShort
@@ -356,7 +358,8 @@ class ReplicaManagerTest {
           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))
+      replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
+        .localReplicaOrException
 
       val producerId = 234L
       val epoch = 5.toShort
@@ -421,7 +424,8 @@ class ReplicaManagerTest {
           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))
+      rm.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true)
+        .localReplicaOrException
 
       // Append a couple of messages.
       for(i <- 1 to 2) {
@@ -493,14 +497,14 @@ class ReplicaManagerTest {
         // the response contains high watermark on the leader before it is updated based
         // on this fetch request
         assertEquals(0, tp0Status.get.highWatermark)
-        assertEquals(None, tp0Status.get.lastStableOffset)
+        assertEquals(Some(0), tp0Status.get.lastStableOffset)
         assertEquals(Errors.NONE, tp0Status.get.error)
         assertTrue(tp0Status.get.records.batches.iterator.hasNext)
 
         val tp1Status = responseStatusMap.get(tp1)
         assertTrue(tp1Status.isDefined)
         assertEquals(0, tp1Status.get.highWatermark)
-        assertEquals(None, tp0Status.get.lastStableOffset)
+        assertEquals(Some(0), tp0Status.get.lastStableOffset)
         assertEquals(Errors.NONE, tp1Status.get.error)
         assertFalse(tp1Status.get.records.batches.iterator.hasNext)
       }
@@ -517,12 +521,12 @@ class ReplicaManagerTest {
         responseCallback = fetchCallback,
         isolationLevel = IsolationLevel.READ_UNCOMMITTED
       )
-      val tp0Replica = replicaManager.getReplica(tp0)
+      val tp0Replica = replicaManager.localReplica(tp0)
       assertTrue(tp0Replica.isDefined)
       assertEquals("hw should be incremented", 1, tp0Replica.get.highWatermark.messageOffset)
 
-      replicaManager.getReplica(tp1)
-      val tp1Replica = replicaManager.getReplica(tp1)
+      replicaManager.localReplica(tp1)
+      val tp1Replica = replicaManager.localReplica(tp1)
       assertTrue(tp1Replica.isDefined)
       assertEquals("hw should not be incremented", 0, tp1Replica.get.highWatermark.messageOffset)
 
@@ -574,57 +578,6 @@ class ReplicaManagerTest {
     EasyMock.verify(mockLogMgr)
   }
 
-  /**
-    * If a partition becomes a follower and the leader is unchanged but no epoch update
-    * has been missed, it should not check for truncation
-    */
-  @Test
-  def testDontBecomeFollowerWhenNoMissedLeaderUpdate() {
-    val topicPartition = 0
-    val followerBrokerId = 0
-    val leaderBrokerId = 1
-    val controllerId = 0
-    var leaderEpoch = 1
-    val aliveBrokerIds = Seq[Integer] (followerBrokerId, leaderBrokerId)
-    val countDownLatch = new CountDownLatch(1)
-
-    // Prepare the mocked components for the test
-    val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(
-      topicPartition, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = false)
-
-    // Initialize partition state to follower, with leader = 1, leaderEpoch = 1
-    val partition = replicaManager.getOrCreatePartition(new TopicPartition(topic, topicPartition))
-    partition.getOrCreateReplica(followerBrokerId)
-    partition.makeFollower(controllerId,
-      leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds),
-      correlationId)
-
-    // Make local partition a follower - because epoch did not change, truncation should not trigger
-    val leaderAndIsrRequest0 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion,
-      controllerId, controllerEpoch,
-      collection.immutable.Map(new TopicPartition(topic, topicPartition) ->
-        leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava,
-      Set(new Node(followerBrokerId, "host1", 0),
-        new Node(leaderBrokerId, "host2", 1)).asJava).build()
-    replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest0,
-      (_, followers) => assertTrue(followers.isEmpty))
-
-    // Make local partition a follower - because epoch increased by only 1 and leader did not change,
-    // truncation should not trigger
-    leaderEpoch += 1
-    val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion,
-      controllerId, controllerEpoch,
-      collection.immutable.Map(new TopicPartition(topic, topicPartition) ->
-        leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava,
-      Set(new Node(followerBrokerId, "host1", 0),
-        new Node(leaderBrokerId, "host2", 1)).asJava).build()
-    replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest1,
-      (_, followers) => assertTrue(followers.isEmpty))
-
-    // Truncation should not have happened
-    EasyMock.verify(mockLogMgr)
-  }
-
   private def prepareReplicaManagerAndLogManager(topicPartition: Int,
                                                  followerBrokerId: Int,
                                                  leaderBrokerId: Int,
@@ -711,14 +664,15 @@ class ReplicaManagerTest {
                                                      quotaManager: ReplicationQuotaManager): ReplicaFetcherManager = {
         new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager) {
 
-          override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = {
+          override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): ReplicaFetcherThread = {
             new ReplicaFetcherThread(s"ReplicaFetcherThread-$fetcherId", fetcherId,
               sourceBroker, config, replicaManager, metrics, time, quota.follower, Some(blockingSend)) {
 
               override def doWork() = {
                 // In case the thread starts before the partition is added by AbstractFetcherManager,
                 // add it here (it's a no-op if already added)
-                addPartitions(Map(new TopicPartition(topic, topicPartition) -> 0L))
+                val initialOffset = OffsetAndEpoch(offset = 0L, leaderEpoch = 1)
+                addPartitions(Map(new TopicPartition(topic, topicPartition) -> initialOffset))
                 super.doWork()
 
                 // Shut the thread down after one iteration to avoid double-counting truncations
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index 7032724..3604385 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -292,8 +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, Optional.of(15), 0)
+          new OffsetsForLeaderEpochRequest.Builder(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion,
+            Map(tp -> new OffsetsForLeaderEpochRequest.PartitionData(Optional.of(15), 0)).asJava)
 
         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 d2f1c0a..ecfbd73 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -89,7 +89,7 @@ class SimpleFetchTest {
       maxLength = fetchSize,
       maxOffset = Some(partitionHW),
       minOneMessage = true,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED))
+      includeAbortedTxns = false))
       .andReturn(FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
         MemoryRecords.withRecords(CompressionType.NONE, recordToHW)
@@ -99,7 +99,7 @@ class SimpleFetchTest {
       maxLength = fetchSize,
       maxOffset = None,
       minOneMessage = true,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED))
+      includeAbortedTxns = false))
       .andReturn(FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
         MemoryRecords.withRecords(CompressionType.NONE, recordToLEO)
@@ -174,12 +174,11 @@ class SimpleFetchTest {
     val readCommittedRecords = replicaManager.readFromLocalLog(
       replicaId = Request.OrdinaryConsumerId,
       fetchOnlyFromLeader = true,
-      readOnlyCommitted = true,
+      fetchIsolation = FetchHighWatermark,
       fetchMaxBytes = Int.MaxValue,
       hardMaxBytesLimit = false,
       readPartitionInfo = fetchInfo,
-      quota = UnboundedQuota,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED).find(_._1 == topicPartition)
+      quota = UnboundedQuota).find(_._1 == topicPartition)
     val firstReadRecord = readCommittedRecords.get._2.info.records.records.iterator.next()
     assertEquals("Reading committed data should return messages only up to high watermark", recordToHW,
       new SimpleRecord(firstReadRecord))
@@ -187,12 +186,11 @@ class SimpleFetchTest {
     val readAllRecords = replicaManager.readFromLocalLog(
       replicaId = Request.OrdinaryConsumerId,
       fetchOnlyFromLeader = true,
-      readOnlyCommitted = false,
+      fetchIsolation = FetchLogEnd,
       fetchMaxBytes = Int.MaxValue,
       hardMaxBytesLimit = false,
       readPartitionInfo = fetchInfo,
-      quota = UnboundedQuota,
-      isolationLevel = IsolationLevel.READ_UNCOMMITTED).find(_._1 == topicPartition)
+      quota = UnboundedQuota).find(_._1 == topicPartition)
 
     val firstRecord = readAllRecords.get._2.info.records.records.iterator.next()
     assertEquals("Reading any data can return messages up to the end of the log", recordToLEO,
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 efc0717..dcb8852 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
@@ -145,7 +145,8 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging {
 
     brokers += createServer(fromProps(createBrokerConfig(101, zkConnect)))
 
-    def leo() = brokers(1).replicaManager.getReplica(tp).get.logEndOffset.messageOffset
+    def leo() = brokers(1).replicaManager.localReplica(tp).get.logEndOffset.messageOffset
+
     TestUtils.createTopic(zkClient, tp.topic, Map(tp.partition -> Seq(101)), brokers)
     producer = createProducer(getBrokerListStrFromServers(brokers), acks = -1)
 
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index df9902f..9974230 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -779,6 +779,28 @@ object TestUtils extends Logging {
     server.replicaManager.getPartition(new TopicPartition(topic, partitionId)).exists(_.leaderReplicaIfLocal.isDefined)
   }
 
+  def findLeaderEpoch(brokerId: Int,
+                      topicPartition: TopicPartition,
+                      servers: Iterable[KafkaServer]): Int = {
+    val leaderServer = servers.find(_.config.brokerId == brokerId)
+    val leaderPartition = leaderServer.flatMap(_.replicaManager.getPartition(topicPartition))
+      .getOrElse(fail(s"Failed to find expected replica on broker $brokerId"))
+    leaderPartition.getLeaderEpoch
+  }
+
+  def findFollowerId(topicPartition: TopicPartition,
+                     servers: Iterable[KafkaServer]): Int = {
+    val followerOpt = servers.find { server =>
+      server.replicaManager.getPartition(topicPartition) match {
+        case Some(partition) => !partition.leaderReplicaIdOpt.contains(server.config.brokerId)
+        case None => false
+      }
+    }
+    followerOpt
+      .map(_.config.brokerId)
+      .getOrElse(fail(s"Unable to locate follower for $topicPartition"))
+  }
+
   /**
     * Wait until all brokers know about each other.
     *
@@ -828,14 +850,36 @@ object TestUtils extends Logging {
     controllerId.getOrElse(fail(s"Controller not elected after $timeout ms"))
   }
 
-  def waitUntilLeaderIsKnown(servers: Seq[KafkaServer], topic: String, partition: Int,
-                             timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Unit = {
-    val tp = new TopicPartition(topic, partition)
-    TestUtils.waitUntilTrue(() =>
-      servers.exists { server =>
+  def awaitLeaderChange(servers: Seq[KafkaServer],
+                        tp: TopicPartition,
+                        oldLeader: Int,
+                        timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
+    def newLeaderExists: Option[Int] = {
+      servers.find { server =>
+        server.config.brokerId != oldLeader &&
+          server.replicaManager.getPartition(tp).exists(_.leaderReplicaIfLocal.isDefined)
+      }.map(_.config.brokerId)
+    }
+
+    TestUtils.waitUntilTrue(() => newLeaderExists.isDefined,
+      s"Did not observe leader change for partition $tp after $timeout ms", waitTime = timeout)
+
+    newLeaderExists.get
+  }
+
+  def waitUntilLeaderIsKnown(servers: Seq[KafkaServer],
+                             tp: TopicPartition,
+                             timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
+    def leaderIfExists: Option[Int] = {
+      servers.find { server =>
         server.replicaManager.getPartition(tp).exists(_.leaderReplicaIfLocal.isDefined)
-      }, s"Partition $tp leaders not made yet after $timeout ms", waitTime = timeout
-    )
+      }.map(_.config.brokerId)
+    }
+
+    TestUtils.waitUntilTrue(() => leaderIfExists.isDefined,
+      s"Partition $tp leaders not made yet after $timeout ms", waitTime = timeout)
+
+    leaderIfExists.get
   }
 
   def writeNonsenseToFile(fileName: File, position: Long, size: Int) {