@@ -176,7 +176,7 @@ public class ReplicaManager {
176176 private final SnapshotContext kvSnapshotContext ;
177177
178178 // remote log manager for remote log storage.
179- private final RemoteLogManager remoteLogManager ;
179+ protected final RemoteLogManager remoteLogManager ;
180180
181181 // for metrics
182182 private final TabletServerMetricGroup serverMetricGroup ;
@@ -217,7 +217,7 @@ public ReplicaManager(
217217 }
218218
219219 @ VisibleForTesting
220- ReplicaManager (
220+ protected ReplicaManager (
221221 Configuration conf ,
222222 Scheduler scheduler ,
223223 LogManager logManager ,
@@ -1101,7 +1101,9 @@ public Map<TableBucket, LogReadResult> readFromLog(
11011101
11021102 FetchLogResultForBucket result ;
11031103 if (replica != null && e instanceof LogOffsetOutOfRangeException ) {
1104- result = handleFetchOutOfRangeException (replica , fetchOffset , e );
1104+ result =
1105+ handleFetchOutOfRangeException (
1106+ replica , fetchOffset , fetchParams .isFromFollower (), e );
11051107 } else {
11061108 result = new FetchLogResultForBucket (tb , ApiError .fromThrowable (e ));
11071109 }
@@ -1113,7 +1115,7 @@ public Map<TableBucket, LogReadResult> readFromLog(
11131115 }
11141116
11151117 private FetchLogResultForBucket handleFetchOutOfRangeException (
1116- Replica replica , long fetchOffset , Exception e ) {
1118+ Replica replica , long fetchOffset , boolean isFromFollower , Exception e ) {
11171119 TableBucket tb = replica .getTableBucket ();
11181120 if (fetchOffset == FetchParams .FETCH_FROM_EARLIEST_OFFSET ) {
11191121 fetchOffset = replica .getLogStartOffset ();
@@ -1131,7 +1133,8 @@ private FetchLogResultForBucket handleFetchOutOfRangeException(
11311133 // of RemoteLogSegment. For client fetcher, it will fetch the log from remote in client.
11321134 // For follower, it can update its local metadata to adjust the next fetch offset.
11331135 else if (canFetchFromRemoteLog (replica , fetchOffset )) {
1134- RemoteLogFetchInfo remoteLogFetchInfo = fetchLogFromRemote (replica , fetchOffset );
1136+ RemoteLogFetchInfo remoteLogFetchInfo =
1137+ fetchLogFromRemote (replica , fetchOffset , isFromFollower );
11351138 if (remoteLogFetchInfo != null ) {
11361139 return new FetchLogResultForBucket (
11371140 tb , remoteLogFetchInfo , replica .getLogHighWatermark ());
@@ -1157,13 +1160,20 @@ private boolean canFetchFromRemoteLog(Replica replica, long fetchOffset) {
11571160 return replica .getLogTablet ().canFetchFromRemoteLog (fetchOffset );
11581161 }
11591162
1160- private @ Nullable RemoteLogFetchInfo fetchLogFromRemote (Replica replica , long fetchOffset ) {
1163+ private @ Nullable RemoteLogFetchInfo fetchLogFromRemote (
1164+ Replica replica , long fetchOffset , boolean isFromFollower ) {
11611165 List <RemoteLogSegment > remoteLogSegmentList =
11621166 remoteLogManager .relevantRemoteLogSegments (replica .getTableBucket (), fetchOffset );
11631167 if (!remoteLogSegmentList .isEmpty ()) {
1164- int firstStartPos =
1165- remoteLogManager .lookupPositionForOffset (
1166- remoteLogSegmentList .get (0 ), fetchOffset );
1168+ // follower does not require firstStartPos, so we do not look up this value.
1169+ // On one hand, this reduces query overhead, and on the other hand,
1170+ // it helps prevent unexpected issues caused by corrupted index files.
1171+ int firstStartPos = -1 ;
1172+ if (!isFromFollower ) {
1173+ firstStartPos =
1174+ remoteLogManager .lookupPositionForOffset (
1175+ remoteLogSegmentList .get (0 ), fetchOffset );
1176+ }
11671177 PhysicalTablePath physicalTablePath = replica .getPhysicalTablePath ();
11681178 FsPath remoteLogTabletDir =
11691179 FlussPaths .remoteLogTabletDir (
0 commit comments