You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bb...@apache.org on 2024/01/23 16:05:17 UTC

(hbase) branch master updated: HBASE-28302 Add tracking of fs read times in ScanMetrics and slow logs (#5622)

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

bbeaudreault pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 3a91c5b6981 HBASE-28302 Add tracking of fs read times in ScanMetrics and slow logs (#5622)
3a91c5b6981 is described below

commit 3a91c5b6981023a11b272f76a502891c3728f9b2
Author: Bryan Beaudreault <bb...@apache.org>
AuthorDate: Tue Jan 23 11:05:11 2024 -0500

    HBASE-28302 Add tracking of fs read times in ScanMetrics and slow logs (#5622)
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
---
 .../hadoop/hbase/client/OnlineLogRecord.java       |  29 ++++--
 .../client/metrics/ServerSideScanMetrics.java      |   4 +
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |   1 +
 .../hadoop/hbase/client/TestOnlineLogRecord.java   |  24 ++---
 .../apache/hadoop/hbase/io/MetricsIOSource.java    |   5 +
 .../hadoop/hbase/io/MetricsIOSourceImpl.java       |   8 ++
 .../main/protobuf/server/region/TooSlowLog.proto   |   2 +
 .../java/org/apache/hadoop/hbase/io/MetricsIO.java |   4 +
 .../org/apache/hadoop/hbase/io/hfile/HFile.java    |   7 +-
 .../apache/hadoop/hbase/io/hfile/HFileBlock.java   |   5 +-
 .../java/org/apache/hadoop/hbase/ipc/RpcCall.java  |   4 +
 .../org/apache/hadoop/hbase/ipc/RpcServer.java     |  14 ++-
 .../org/apache/hadoop/hbase/ipc/ServerCall.java    |  11 ++
 .../hadoop/hbase/namequeues/RpcLogDetails.java     |   9 +-
 .../hbase/namequeues/impl/SlowLogQueueService.java |   4 +-
 .../hadoop/hbase/regionserver/RSRpcServices.java   |   3 +
 .../TestServerSideScanMetricsFromClientSide.java   |  32 +++++-
 .../hbase/namequeues/TestNamedQueueRecorder.java   |  18 +++-
 .../hadoop/hbase/namequeues/TestRpcLogDetails.java |  12 ++-
 .../hadoop/hbase/namequeues/TestTooLargeLog.java   |   2 +-
 .../store/region/TestRegionProcedureStore.java     |  10 ++
 .../hadoop/hbase/thrift2/ThriftUtilities.java      |   1 +
 .../hbase/thrift2/generated/TOnlineLogRecord.java  | 111 +++++++++++++++++++--
 .../org/apache/hadoop/hbase/thrift2/hbase.thrift   |   1 +
 24 files changed, 275 insertions(+), 46 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
index d9fd51e80a9..26979129cf1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java
@@ -81,6 +81,7 @@ final public class OnlineLogRecord extends LogEntry {
   private final int queueTime;
   private final long responseSize;
   private final long blockBytesScanned;
+  private final long fsReadTime;
   private final String clientAddress;
   private final String serverClass;
   private final String methodName;
@@ -120,6 +121,10 @@ final public class OnlineLogRecord extends LogEntry {
     return blockBytesScanned;
   }
 
+  public long getFsReadTime() {
+    return fsReadTime;
+  }
+
   public String getClientAddress() {
     return clientAddress;
   }
@@ -178,16 +183,18 @@ final public class OnlineLogRecord extends LogEntry {
   }
 
   OnlineLogRecord(final long startTime, final int processingTime, final int queueTime,
-    final long responseSize, final long blockBytesScanned, final String clientAddress,
-    final String serverClass, final String methodName, final String callDetails, final String param,
-    final String regionName, final String userName, final int multiGetsCount,
-    final int multiMutationsCount, final int multiServiceCalls, final Scan scan,
-    final Map<String, byte[]> requestAttributes, final Map<String, byte[]> connectionAttributes) {
+    final long responseSize, final long blockBytesScanned, final long fsReadTime,
+    final String clientAddress, final String serverClass, final String methodName,
+    final String callDetails, final String param, final String regionName, final String userName,
+    final int multiGetsCount, final int multiMutationsCount, final int multiServiceCalls,
+    final Scan scan, final Map<String, byte[]> requestAttributes,
+    final Map<String, byte[]> connectionAttributes) {
     this.startTime = startTime;
     this.processingTime = processingTime;
     this.queueTime = queueTime;
     this.responseSize = responseSize;
     this.blockBytesScanned = blockBytesScanned;
+    this.fsReadTime = fsReadTime;
     this.clientAddress = clientAddress;
     this.serverClass = serverClass;
     this.methodName = methodName;
@@ -209,6 +216,7 @@ final public class OnlineLogRecord extends LogEntry {
     private int queueTime;
     private long responseSize;
     private long blockBytesScanned;
+    private long fsReadTime;
     private String clientAddress;
     private String serverClass;
     private String methodName;
@@ -251,6 +259,11 @@ final public class OnlineLogRecord extends LogEntry {
       return this;
     }
 
+    public OnlineLogRecordBuilder setFsReadTime(long fsReadTime) {
+      this.fsReadTime = fsReadTime;
+      return this;
+    }
+
     public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
       this.clientAddress = clientAddress;
       return this;
@@ -319,9 +332,9 @@ final public class OnlineLogRecord extends LogEntry {
 
     public OnlineLogRecord build() {
       return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
-        blockBytesScanned, clientAddress, serverClass, methodName, callDetails, param, regionName,
-        userName, multiGetsCount, multiMutationsCount, multiServiceCalls, scan, requestAttributes,
-        connectionAttributes);
+        blockBytesScanned, fsReadTime, clientAddress, serverClass, methodName, callDetails, param,
+        regionName, userName, multiGetsCount, multiMutationsCount, multiServiceCalls, scan,
+        requestAttributes, connectionAttributes);
     }
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
index cf730501be0..ff83584ccb4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
@@ -50,6 +50,8 @@ public class ServerSideScanMetrics {
 
   public static final String BLOCK_BYTES_SCANNED_KEY_METRIC_NAME = "BLOCK_BYTES_SCANNED";
 
+  public static final String FS_READ_TIME_METRIC_NAME = "FS_READ_TIME";
+
   /**
    * number of rows filtered during scan RPC
    */
@@ -65,6 +67,8 @@ public class ServerSideScanMetrics {
   public final AtomicLong countOfBlockBytesScanned =
     createCounter(BLOCK_BYTES_SCANNED_KEY_METRIC_NAME);
 
+  public final AtomicLong fsReadTime = createCounter(FS_READ_TIME_METRIC_NAME);
+
   public void setCounter(String counterName, long value) {
     AtomicLong c = this.counters.get(counterName);
     if (c != null) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 598ad932e67..e50b54e8eb0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -3462,6 +3462,7 @@ public final class ProtobufUtil {
         .setQueueTime(slowLogPayload.getQueueTime()).setRegionName(slowLogPayload.getRegionName())
         .setResponseSize(slowLogPayload.getResponseSize())
         .setBlockBytesScanned(slowLogPayload.getBlockBytesScanned())
+        .setFsReadTime(slowLogPayload.getFsReadTime())
         .setServerClass(slowLogPayload.getServerClass()).setStartTime(slowLogPayload.getStartTime())
         .setUserName(slowLogPayload.getUserName())
         .setRequestAttributes(convertNameBytesPairsToMap(slowLogPayload.getRequestAttributeList()))
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java
index fe753973ae2..a16993d5659 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java
@@ -46,15 +46,15 @@ public class TestOnlineLogRecord {
     scan.withStopRow(Bytes.toBytes(456));
     String expectedOutput = "{\n" + "  \"startTime\": 1,\n" + "  \"processingTime\": 2,\n"
       + "  \"queueTime\": 3,\n" + "  \"responseSize\": 4,\n" + "  \"blockBytesScanned\": 5,\n"
-      + "  \"multiGetsCount\": 6,\n" + "  \"multiMutationsCount\": 7,\n" + "  \"scan\": {\n"
-      + "    \"startRow\": \"\\\\x00\\\\x00\\\\x00{\",\n"
+      + "  \"fsReadTime\": 6,\n" + "  \"multiGetsCount\": 6,\n" + "  \"multiMutationsCount\": 7,\n"
+      + "  \"scan\": {\n" + "    \"startRow\": \"\\\\x00\\\\x00\\\\x00{\",\n"
       + "    \"stopRow\": \"\\\\x00\\\\x00\\\\x01\\\\xC8\",\n" + "    \"batch\": -1,\n"
       + "    \"cacheBlocks\": true,\n" + "    \"totalColumns\": 0,\n"
       + "    \"maxResultSize\": -1,\n" + "    \"families\": {},\n" + "    \"caching\": -1,\n"
       + "    \"maxVersions\": 1,\n" + "    \"timeRange\": [\n" + "      0,\n"
       + "      9223372036854775807\n" + "    ]\n" + "  }\n" + "}";
-    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null,
-      6, 7, 0, scan, Collections.emptyMap(), Collections.emptyMap());
+    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null,
+      null, 6, 7, 0, scan, Collections.emptyMap(), Collections.emptyMap());
     String actualOutput = o.toJsonPrettyPrint();
     System.out.println(actualOutput);
     Assert.assertEquals(actualOutput, expectedOutput);
@@ -67,8 +67,8 @@ public class TestOnlineLogRecord {
     Set<String> expectedOutputs =
       ImmutableSet.<String> builder().add("requestAttributes").add("\"r\": \"1\"")
         .add("\"2\": \"\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\"").build();
-    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null,
-      6, 7, 0, null, requestAttributes, Collections.emptyMap());
+    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null,
+      null, 6, 7, 0, null, requestAttributes, Collections.emptyMap());
     String actualOutput = o.toJsonPrettyPrint();
     System.out.println(actualOutput);
     expectedOutputs.forEach(expected -> Assert.assertTrue(actualOutput.contains(expected)));
@@ -76,8 +76,8 @@ public class TestOnlineLogRecord {
 
   @Test
   public void itOmitsEmptyRequestAttributes() {
-    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null,
-      6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap());
+    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null,
+      null, 6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap());
     String actualOutput = o.toJsonPrettyPrint();
     System.out.println(actualOutput);
     Assert.assertFalse(actualOutput.contains("requestAttributes"));
@@ -90,8 +90,8 @@ public class TestOnlineLogRecord {
     Set<String> expectedOutputs =
       ImmutableSet.<String> builder().add("connectionAttributes").add("\"c\": \"1\"")
         .add("\"2\": \"\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\"").build();
-    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null,
-      6, 7, 0, null, Collections.emptyMap(), connectionAttributes);
+    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null,
+      null, 6, 7, 0, null, Collections.emptyMap(), connectionAttributes);
     String actualOutput = o.toJsonPrettyPrint();
     System.out.println(actualOutput);
     expectedOutputs.forEach(expected -> Assert.assertTrue(actualOutput.contains(expected)));
@@ -99,8 +99,8 @@ public class TestOnlineLogRecord {
 
   @Test
   public void itOmitsEmptyConnectionAttributes() {
-    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null,
-      6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap());
+    OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null,
+      null, 6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap());
     String actualOutput = o.toJsonPrettyPrint();
     System.out.println(actualOutput);
     Assert.assertFalse(actualOutput.contains("connectionAttributes"));
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java
index af7e87483d1..5b4fc4b2c69 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java
@@ -47,6 +47,9 @@ public interface MetricsIOSource extends BaseSource {
   String FS_PREAD_TIME_HISTO_KEY = "fsPReadTime";
   String FS_WRITE_HISTO_KEY = "fsWriteTime";
 
+  String SLOW_FS_READS_KEY = "fsSlowReadsCount";
+  String SLOW_FS_READS_DESC = "Number of HFile reads which were slower than a configured threshold";
+
   String CHECKSUM_FAILURES_KEY = "fsChecksumFailureCount";
 
   String FS_READ_TIME_HISTO_DESC =
@@ -76,4 +79,6 @@ public interface MetricsIOSource extends BaseSource {
    * @param t time it took, in milliseconds
    */
   void updateFsWriteTime(long t);
+
+  void incrSlowFsRead();
 }
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java
index 6ef5d180cd5..5aca9a3d84b 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
@@ -32,6 +33,7 @@ public class MetricsIOSourceImpl extends BaseSourceImpl implements MetricsIOSour
   private final MetricHistogram fsReadTimeHisto;
   private final MetricHistogram fsPReadTimeHisto;
   private final MetricHistogram fsWriteTimeHisto;
+  private final MutableFastCounter fsSlowReads;
 
   public MetricsIOSourceImpl(MetricsIOWrapper wrapper) {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, wrapper);
@@ -49,6 +51,7 @@ public class MetricsIOSourceImpl extends BaseSourceImpl implements MetricsIOSour
       getMetricsRegistry().newTimeHistogram(FS_PREAD_TIME_HISTO_KEY, FS_PREAD_TIME_HISTO_DESC);
     fsWriteTimeHisto =
       getMetricsRegistry().newTimeHistogram(FS_WRITE_HISTO_KEY, FS_WRITE_TIME_HISTO_DESC);
+    fsSlowReads = getMetricsRegistry().newCounter(SLOW_FS_READS_KEY, SLOW_FS_READS_DESC, 0L);
   }
 
   @Override
@@ -66,6 +69,11 @@ public class MetricsIOSourceImpl extends BaseSourceImpl implements MetricsIOSour
     fsWriteTimeHisto.add(t);
   }
 
+  @Override
+  public void incrSlowFsRead() {
+    fsSlowReads.incr();
+  }
+
   @Override
   public void getMetrics(MetricsCollector metricsCollector, boolean all) {
     MetricsRecordBuilder mrb = metricsCollector.addRecord(metricsName);
diff --git a/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto b/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto
index 4c275948b27..80b984e999c 100644
--- a/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto
@@ -53,6 +53,8 @@ message SlowLogPayload {
   repeated NameBytesPair connection_attribute = 18;
   repeated NameBytesPair request_attribute = 19;
 
+  optional int64 fs_read_time = 20;
+
   // SLOW_LOG is RPC call slow in nature whereas LARGE_LOG is RPC call quite large.
   // Majority of times, slow logs are also large logs and hence, ALL is combination of
   // both
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java
index 58e6f7d01b7..4d2437d418b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java
@@ -75,4 +75,8 @@ public class MetricsIO {
   public void updateFsWriteTime(long t) {
     source.updateFsWriteTime(t);
   }
+
+  public void incrSlowFsRead() {
+    source.incrSlowFsRead();
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 207c9986651..84fe9387d6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.io.MetricsIO;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
+import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.ShipperListener;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
@@ -187,12 +188,16 @@ public final class HFile {
     return CHECKSUM_FAILURES.sum();
   }
 
-  public static final void updateReadLatency(long latencyMillis, boolean pread) {
+  public static final void updateReadLatency(long latencyMillis, boolean pread, boolean tooSlow) {
+    RpcServer.getCurrentCall().ifPresent(call -> call.updateFsReadTime(latencyMillis));
     if (pread) {
       MetricsIO.getInstance().updateFsPreadTime(latencyMillis);
     } else {
       MetricsIO.getInstance().updateFsReadTime(latencyMillis);
     }
+    if (tooSlow) {
+      MetricsIO.getInstance().incrSlowFsRead();
+    }
   }
 
   public static final void updateWriteLatency(long latencyMillis) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index a3ead34730f..47c20b691b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1826,8 +1826,9 @@ public class HFileBlock implements Cacheable {
         int sizeWithoutChecksum = curBlock.getInt(Header.ON_DISK_DATA_SIZE_WITH_HEADER_INDEX);
         curBlock.limit(sizeWithoutChecksum);
         long duration = EnvironmentEdgeManager.currentTime() - startTime;
+        boolean tooSlow = this.readWarnTime >= 0 && duration > this.readWarnTime;
         if (updateMetrics) {
-          HFile.updateReadLatency(duration, pread);
+          HFile.updateReadLatency(duration, pread, tooSlow);
         }
         // The onDiskBlock will become the headerAndDataBuffer for this block.
         // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already
@@ -1839,7 +1840,7 @@ public class HFileBlock implements Cacheable {
           hFileBlock.sanityCheckUncompressed();
         }
         LOG.trace("Read {} in {} ms", hFileBlock, duration);
-        if (!LOG.isTraceEnabled() && this.readWarnTime >= 0 && duration > this.readWarnTime) {
+        if (!LOG.isTraceEnabled() && tooSlow) {
           LOG.warn("Read Block Slow: read {} cost {} ms, threshold = {} ms", hFileBlock, duration,
             this.readWarnTime);
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java
index 260d6e1a980..2d06aa7c47a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java
@@ -132,4 +132,8 @@ public interface RpcCall extends RpcCallContext {
 
   /** Returns A short string format of this call without possibly lengthy params */
   String toShortString();
+
+  void updateFsReadTime(long latencyMillis);
+
+  long getFsReadTime();
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index d3ec4ff8c73..0876a1fd55f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -444,14 +444,17 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
       int totalTime = (int) (endTime - receiveTime);
       if (LOG.isTraceEnabled()) {
         LOG.trace(
-          "{}, response: {}, receiveTime: {}, queueTime: {}, processingTime: {}, totalTime: {}",
+          "{}, response: {}, receiveTime: {}, queueTime: {}, processingTime: {}, "
+            + "totalTime: {}, fsReadTime: {}",
           CurCall.get().toString(), TextFormat.shortDebugString(result),
-          CurCall.get().getReceiveTime(), qTime, processingTime, totalTime);
+          CurCall.get().getReceiveTime(), qTime, processingTime, totalTime,
+          CurCall.get().getFsReadTime());
       }
       // Use the raw request call size for now.
       long requestSize = call.getSize();
       long responseSize = result.getSerializedSize();
       long responseBlockSize = call.getBlockBytesScanned();
+      long fsReadTime = call.getFsReadTime();
       if (call.isClientCellBlockSupported()) {
         // Include the payload size in HBaseRpcController
         responseSize += call.getResponseCellSize();
@@ -472,13 +475,13 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
         // note that large responses will often also be slow.
         logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
           tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize,
-          responseBlockSize, userName);
+          responseBlockSize, fsReadTime, userName);
         if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) {
           // send logs to ring buffer owned by slowLogRecorder
           final String className =
             server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
           this.namedQueueRecorder.addRecord(new RpcLogDetails(call, param, status.getClient(),
-            responseSize, responseBlockSize, className, tooSlow, tooLarge));
+            responseSize, responseBlockSize, fsReadTime, className, tooSlow, tooLarge));
         }
       }
       return new Pair<>(result, controller.cellScanner());
@@ -522,7 +525,7 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
    */
   void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow,
     String clientAddress, long startTime, int processingTime, int qTime, long responseSize,
-    long blockBytesScanned, String userName) {
+    long blockBytesScanned, long fsReadTime, String userName) {
     final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
     // base information that is reported regardless of type of call
     Map<String, Object> responseInfo = new HashMap<>();
@@ -531,6 +534,7 @@ public abstract class RpcServer implements RpcServerInterface, ConfigurationObse
     responseInfo.put("queuetimems", qTime);
     responseInfo.put("responsesize", responseSize);
     responseInfo.put("blockbytesscanned", blockBytesScanned);
+    responseInfo.put("fsreadtime", fsReadTime);
     responseInfo.put("client", clientAddress);
     responseInfo.put("class", className);
     responseInfo.put("method", methodName);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
index ed688977b96..a2c578fd666 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
@@ -99,6 +99,7 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
 
   private long responseCellSize = 0;
   private long responseBlockSize = 0;
+  private long fsReadTimeMillis = 0;
   // cumulative size of serialized exceptions
   private long exceptionSize = 0;
   private final boolean retryImmediatelySupported;
@@ -567,4 +568,14 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
   public synchronized BufferChain getResponse() {
     return response;
   }
+
+  @Override
+  public void updateFsReadTime(long latencyMillis) {
+    fsReadTimeMillis += latencyMillis;
+  }
+
+  @Override
+  public long getFsReadTime() {
+    return fsReadTimeMillis;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java
index 235d82302d6..263fff66a73 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java
@@ -42,6 +42,7 @@ public class RpcLogDetails extends NamedQueuePayload {
   private final String clientAddress;
   private final long responseSize;
   private final long blockBytesScanned;
+  private final long fsReadTime;
   private final String className;
   private final boolean isSlowLog;
   private final boolean isLargeLog;
@@ -49,12 +50,14 @@ public class RpcLogDetails extends NamedQueuePayload {
   private final Map<String, byte[]> requestAttributes;
 
   public RpcLogDetails(RpcCall rpcCall, Message param, String clientAddress, long responseSize,
-    long blockBytesScanned, String className, boolean isSlowLog, boolean isLargeLog) {
+    long blockBytesScanned, long fsReadTime, String className, boolean isSlowLog,
+    boolean isLargeLog) {
     super(SLOW_LOG_EVENT);
     this.rpcCall = rpcCall;
     this.clientAddress = clientAddress;
     this.responseSize = responseSize;
     this.blockBytesScanned = blockBytesScanned;
+    this.fsReadTime = fsReadTime;
     this.className = className;
     this.isSlowLog = isSlowLog;
     this.isLargeLog = isLargeLog;
@@ -92,6 +95,10 @@ public class RpcLogDetails extends NamedQueuePayload {
     return blockBytesScanned;
   }
 
+  public long getFsReadTime() {
+    return fsReadTime;
+  }
+
   public String getClassName() {
     return className;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java
index fb29b8563ef..ea4e286bf43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java
@@ -124,6 +124,7 @@ public class SlowLogQueueService implements NamedQueueService {
     final String clientAddress = rpcLogDetails.getClientAddress();
     final long responseSize = rpcLogDetails.getResponseSize();
     final long blockBytesScanned = rpcLogDetails.getBlockBytesScanned();
+    final long fsReadTime = rpcLogDetails.getFsReadTime();
     final String className = rpcLogDetails.getClassName();
     final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails);
     if (type == null) {
@@ -168,7 +169,8 @@ public class SlowLogQueueService implements NamedQueueService {
       .setProcessingTime(processingTime).setQueueTime(qTime)
       .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY)
       .setResponseSize(responseSize).setBlockBytesScanned(blockBytesScanned)
-      .setServerClass(className).setStartTime(startTime).setType(type).setUserName(userName)
+      .setFsReadTime(fsReadTime).setServerClass(className).setStartTime(startTime).setType(type)
+      .setUserName(userName)
       .addAllRequestAttribute(buildNameBytesPairs(rpcLogDetails.getRequestAttributes()))
       .addAllConnectionAttribute(buildNameBytesPairs(rpcLogDetails.getConnectionAttributes()));
     if (slowLogParams != null && slowLogParams.getScan() != null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 0fe6f6476a6..05d7c2e5605 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -3454,6 +3454,9 @@ public class RSRpcServices extends HBaseRpcServicesBase<HRegionServer>
           // from block size progress before writing into the response
           scannerContext.getMetrics().countOfBlockBytesScanned
             .set(scannerContext.getBlockSizeProgress());
+          if (rpcCall != null) {
+            scannerContext.getMetrics().fsReadTime.set(rpcCall.getFsReadTime());
+          }
           Map<String, Long> metrics = scannerContext.getMetrics().getMetricsMap();
           ScanMetrics.Builder metricBuilder = ScanMetrics.newBuilder();
           NameInt64Pair.Builder pairBuilder = NameInt64Pair.newBuilder();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
index b80cd207683..6ac87ce0302 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -195,6 +196,18 @@ public class TestServerSideScanMetricsFromClientSide {
     }
   }
 
+  @Test
+  public void testFsReadTimeMetric() throws Exception {
+    // write some new puts and flush, as an easy way to ensure the read blocks are not cached
+    // so that we go into the fs write code path
+    List<Put> puts = createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE);
+    TABLE.put(puts);
+    TEST_UTIL.flush(TABLE_NAME);
+    Scan scan = new Scan();
+    scan.setScanMetricsEnabled(true);
+    testMetric(scan, ServerSideScanMetrics.FS_READ_TIME_METRIC_NAME, 0, CompareOperator.GREATER);
+  }
+
   private void testRowsSeenMetric(Scan baseScan) throws Exception {
     Scan scan;
     scan = new Scan(baseScan);
@@ -333,6 +346,11 @@ public class TestServerSideScanMetricsFromClientSide {
    * @throws Exception on unexpected failure
    */
   private void testMetric(Scan scan, String metricKey, long expectedValue) throws Exception {
+    testMetric(scan, metricKey, expectedValue, CompareOperator.EQUAL);
+  }
+
+  private void testMetric(Scan scan, String metricKey, long expectedValue,
+    CompareOperator compareOperator) throws Exception {
     assertTrue("Scan should be configured to record metrics", scan.isScanMetricsEnabled());
     ResultScanner scanner = TABLE.getScanner(scan);
     // Iterate through all the results
@@ -341,11 +359,17 @@ public class TestServerSideScanMetricsFromClientSide {
     }
     scanner.close();
     ScanMetrics metrics = scanner.getScanMetrics();
-    assertTrue("Metrics are null", metrics != null);
+    assertNotNull("Metrics are null", metrics);
     assertTrue("Metric : " + metricKey + " does not exist", metrics.hasCounter(metricKey));
     final long actualMetricValue = metrics.getCounter(metricKey).get();
-    assertEquals(
-      "Metric: " + metricKey + " Expected: " + expectedValue + " Actual: " + actualMetricValue,
-      expectedValue, actualMetricValue);
+    if (compareOperator == CompareOperator.EQUAL) {
+      assertEquals(
+        "Metric: " + metricKey + " Expected: " + expectedValue + " Actual: " + actualMetricValue,
+        expectedValue, actualMetricValue);
+    } else {
+      assertTrue(
+        "Metric: " + metricKey + " Expected: > " + expectedValue + " Actual: " + actualMetricValue,
+        actualMetricValue > expectedValue);
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java
index 35a1757115c..00953353187 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java
@@ -667,13 +667,13 @@ public class TestNamedQueueRecorder {
   static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className,
     int forcedParamIndex) {
     RpcCall rpcCall = getRpcCall(userName, forcedParamIndex);
-    return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true,
+    return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, 0, className, true,
       true);
   }
 
   static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className) {
     RpcCall rpcCall = getRpcCall(userName);
-    return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true,
+    return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, 0, className, true,
       true);
   }
 
@@ -685,8 +685,8 @@ public class TestNamedQueueRecorder {
   private RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className,
     boolean isSlowLog, boolean isLargeLog) {
     RpcCall rpcCall = getRpcCall(userName);
-    return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, isSlowLog,
-      isLargeLog);
+    return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, 0, className,
+      isSlowLog, isLargeLog);
   }
 
   private static RpcCall getRpcCall(String userName) {
@@ -859,6 +859,16 @@ public class TestNamedQueueRecorder {
       @Override
       public void incrementResponseExceptionSize(long exceptionSize) {
       }
+
+      @Override
+      public void updateFsReadTime(long latencyMillis) {
+
+      }
+
+      @Override
+      public long getFsReadTime() {
+        return 0;
+      }
     };
     return rpcCall;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java
index 8a93f2d0ff5..67d8a257909 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java
@@ -80,7 +80,7 @@ public class TestRpcLogDetails {
     ProtobufUtil.mergeFrom(messageBuilder, cis, buffer.capacity());
     Message message = messageBuilder.build();
     RpcLogDetails rpcLogDetails =
-      new RpcLogDetails(getRpcCall(message), message, null, 0L, 0L, null, true, false);
+      new RpcLogDetails(getRpcCall(message), message, null, 0L, 0L, 0, null, true, false);
 
     // log's scan should be equal
     ClientProtos.Scan logScan = ((ClientProtos.ScanRequest) rpcLogDetails.getParam()).getScan();
@@ -258,6 +258,16 @@ public class TestRpcLogDetails {
       @Override
       public void incrementResponseExceptionSize(long exceptionSize) {
       }
+
+      @Override
+      public void updateFsReadTime(long latencyMillis) {
+
+      }
+
+      @Override
+      public long getFsReadTime() {
+        return 0;
+      }
     };
     return rpcCall;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java
index da3d9754764..fdc3e288bfe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java
@@ -118,6 +118,6 @@ public class TestTooLargeLog {
       record.getBlockBytesScanned() >= 100);
     assertTrue("expected " + record.getResponseSize() + " to be < 100",
       record.getResponseSize() < 100);
-
+    assertTrue("expected " + record.getFsReadTime() + " to be > 0", record.getFsReadTime() > 0);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java
index 305f0e29e95..d069c2560a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java
@@ -320,6 +320,16 @@ public class TestRegionProcedureStore extends RegionProcedureStoreTestBase {
       @Override
       public void incrementResponseExceptionSize(long exceptionSize) {
       }
+
+      @Override
+      public void updateFsReadTime(long latencyMillis) {
+
+      }
+
+      @Override
+      public long getFsReadTime() {
+        return 0;
+      }
     };
   }
 }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
index 76bc96df05c..a02f944e12a 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
@@ -1647,6 +1647,7 @@ public final class ThriftUtilities {
       tOnlineLogRecord.setRegionName(slowLogRecord.getRegionName());
       tOnlineLogRecord.setResponseSize(slowLogRecord.getResponseSize());
       tOnlineLogRecord.setBlockBytesScanned(slowLogRecord.getBlockBytesScanned());
+      tOnlineLogRecord.setFsReadTime(slowLogRecord.getFsReadTime());
       tOnlineLogRecord.setServerClass(slowLogRecord.getServerClass());
       tOnlineLogRecord.setStartTime(slowLogRecord.getStartTime());
       tOnlineLogRecord.setUserName(slowLogRecord.getUserName());
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java
index 672b8b96d55..c3d6cba7ad2 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.client.OnlineLogRecord
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2023-02-04")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2024-01-12")
 public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecord, TOnlineLogRecord._Fields>, java.io.Serializable, Cloneable, Comparable<TOnlineLogRecord> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOnlineLogRecord");
 
@@ -30,6 +30,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
   private static final org.apache.thrift.protocol.TField MULTI_SERVICE_CALLS_FIELD_DESC = new org.apache.thrift.protocol.TField("multiServiceCalls", org.apache.thrift.protocol.TType.I32, (short)13);
   private static final org.apache.thrift.protocol.TField REGION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("regionName", org.apache.thrift.protocol.TType.STRING, (short)14);
   private static final org.apache.thrift.protocol.TField BLOCK_BYTES_SCANNED_FIELD_DESC = new org.apache.thrift.protocol.TField("blockBytesScanned", org.apache.thrift.protocol.TType.I64, (short)15);
+  private static final org.apache.thrift.protocol.TField FS_READ_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("fsReadTime", org.apache.thrift.protocol.TType.I64, (short)16);
 
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TOnlineLogRecordStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TOnlineLogRecordTupleSchemeFactory();
@@ -49,6 +50,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
   public int multiServiceCalls; // required
   public @org.apache.thrift.annotation.Nullable java.lang.String regionName; // optional
   public long blockBytesScanned; // optional
+  public long fsReadTime; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -66,7 +68,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
     MULTI_MUTATIONS_COUNT((short)12, "multiMutationsCount"),
     MULTI_SERVICE_CALLS((short)13, "multiServiceCalls"),
     REGION_NAME((short)14, "regionName"),
-    BLOCK_BYTES_SCANNED((short)15, "blockBytesScanned");
+    BLOCK_BYTES_SCANNED((short)15, "blockBytesScanned"),
+    FS_READ_TIME((short)16, "fsReadTime");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -112,6 +115,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
           return REGION_NAME;
         case 15: // BLOCK_BYTES_SCANNED
           return BLOCK_BYTES_SCANNED;
+        case 16: // FS_READ_TIME
+          return FS_READ_TIME;
         default:
           return null;
       }
@@ -161,8 +166,9 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
   private static final int __MULTIMUTATIONSCOUNT_ISSET_ID = 5;
   private static final int __MULTISERVICECALLS_ISSET_ID = 6;
   private static final int __BLOCKBYTESSCANNED_ISSET_ID = 7;
-  private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.BLOCK_BYTES_SCANNED};
+  private static final int __FSREADTIME_ISSET_ID = 8;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.BLOCK_BYTES_SCANNED,_Fields.FS_READ_TIME};
   public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -196,6 +202,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.BLOCK_BYTES_SCANNED, new org.apache.thrift.meta_data.FieldMetaData("blockBytesScanned", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FS_READ_TIME, new org.apache.thrift.meta_data.FieldMetaData("fsReadTime", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOnlineLogRecord.class, metaDataMap);
   }
@@ -275,6 +283,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
       this.regionName = other.regionName;
     }
     this.blockBytesScanned = other.blockBytesScanned;
+    this.fsReadTime = other.fsReadTime;
   }
 
   public TOnlineLogRecord deepCopy() {
@@ -306,6 +315,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
     this.regionName = null;
     setBlockBytesScannedIsSet(false);
     this.blockBytesScanned = 0;
+    setFsReadTimeIsSet(false);
+    this.fsReadTime = 0;
   }
 
   public long getStartTime() {
@@ -667,6 +678,29 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
     __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __BLOCKBYTESSCANNED_ISSET_ID, value);
   }
 
+  public long getFsReadTime() {
+    return this.fsReadTime;
+  }
+
+  public TOnlineLogRecord setFsReadTime(long fsReadTime) {
+    this.fsReadTime = fsReadTime;
+    setFsReadTimeIsSet(true);
+    return this;
+  }
+
+  public void unsetFsReadTime() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FSREADTIME_ISSET_ID);
+  }
+
+  /** Returns true if field fsReadTime is set (has been assigned a value) and false otherwise */
+  public boolean isSetFsReadTime() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FSREADTIME_ISSET_ID);
+  }
+
+  public void setFsReadTimeIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FSREADTIME_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
     switch (field) {
     case START_TIME:
@@ -789,6 +823,14 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
       }
       break;
 
+    case FS_READ_TIME:
+      if (value == null) {
+        unsetFsReadTime();
+      } else {
+        setFsReadTime((java.lang.Long)value);
+      }
+      break;
+
     }
   }
 
@@ -840,6 +882,9 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
     case BLOCK_BYTES_SCANNED:
       return getBlockBytesScanned();
 
+    case FS_READ_TIME:
+      return getFsReadTime();
+
     }
     throw new java.lang.IllegalStateException();
   }
@@ -881,6 +926,8 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
       return isSetRegionName();
     case BLOCK_BYTES_SCANNED:
       return isSetBlockBytesScanned();
+    case FS_READ_TIME:
+      return isSetFsReadTime();
     }
     throw new java.lang.IllegalStateException();
   }
@@ -1033,6 +1080,15 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
         return false;
     }
 
+    boolean this_present_fsReadTime = true && this.isSetFsReadTime();
+    boolean that_present_fsReadTime = true && that.isSetFsReadTime();
+    if (this_present_fsReadTime || that_present_fsReadTime) {
+      if (!(this_present_fsReadTime && that_present_fsReadTime))
+        return false;
+      if (this.fsReadTime != that.fsReadTime)
+        return false;
+    }
+
     return true;
   }
 
@@ -1086,6 +1142,10 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
     if (isSetBlockBytesScanned())
       hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(blockBytesScanned);
 
+    hashCode = hashCode * 8191 + ((isSetFsReadTime()) ? 131071 : 524287);
+    if (isSetFsReadTime())
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(fsReadTime);
+
     return hashCode;
   }
 
@@ -1247,6 +1307,16 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
         return lastComparison;
       }
     }
+    lastComparison = java.lang.Boolean.compare(isSetFsReadTime(), other.isSetFsReadTime());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFsReadTime()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fsReadTime, other.fsReadTime);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1359,6 +1429,12 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
       sb.append(this.blockBytesScanned);
       first = false;
     }
+    if (isSetFsReadTime()) {
+      if (!first) sb.append(", ");
+      sb.append("fsReadTime:");
+      sb.append(this.fsReadTime);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1549,6 +1625,14 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 16: // FS_READ_TIME
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.fsReadTime = iprot.readI64();
+              struct.setFsReadTimeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1648,6 +1732,11 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
         oprot.writeI64(struct.blockBytesScanned);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetFsReadTime()) {
+        oprot.writeFieldBegin(FS_READ_TIME_FIELD_DESC);
+        oprot.writeI64(struct.fsReadTime);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1685,13 +1774,19 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
       if (struct.isSetBlockBytesScanned()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetFsReadTime()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetRegionName()) {
         oprot.writeString(struct.regionName);
       }
       if (struct.isSetBlockBytesScanned()) {
         oprot.writeI64(struct.blockBytesScanned);
       }
+      if (struct.isSetFsReadTime()) {
+        oprot.writeI64(struct.fsReadTime);
+      }
     }
 
     @Override
@@ -1723,7 +1818,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
       struct.setMultiMutationsCountIsSet(true);
       struct.multiServiceCalls = iprot.readI32();
       struct.setMultiServiceCallsIsSet(true);
-      java.util.BitSet incoming = iprot.readBitSet(2);
+      java.util.BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.regionName = iprot.readString();
         struct.setRegionNameIsSet(true);
@@ -1732,6 +1827,10 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase<TOnlineLogRecor
         struct.blockBytesScanned = iprot.readI64();
         struct.setBlockBytesScannedIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.fsReadTime = iprot.readI64();
+        struct.setFsReadTimeIsSet(true);
+      }
     }
   }
 
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
index a32f266cf31..ed3fdf32b97 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
@@ -499,6 +499,7 @@ struct TOnlineLogRecord {
   13: required i32 multiServiceCalls
   14: optional string regionName
   15: optional i64 blockBytesScanned
+  16: optional i64 fsReadTime
 }
 
 //