divijvaidya commented on code in PR #14425: URL: https://github.com/apache/kafka/pull/14425#discussion_r1336950457
########## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ########## @@ -697,12 +711,22 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException this.cancel(); } catch (InterruptedException | RetriableException ex) { throw ex; + } catch (CorruptIndexException ex) { + logger.error("Error occurred while copying log segments. Index appeared to be corrupted for partition: {} ", topicIdPartition, ex); + segmentCopyFailures++; Review Comment: We can re-create index from the segment in case it is corrupted. This would prevent need of any operator intervention. It will block uploading to TS for a time duration during which index is being reconstructed but the trade-off in favour of self-recovery is worth it in my opinion. ########## core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java: ########## @@ -1706,6 +1758,279 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { } } + @Test + void testCorruptedTimeIndex() throws Exception { Review Comment: There is lot of duplicated code amongst the three tests added here. Can we use `@ParameterizedTest` and combine them together? ########## storage/src/main/java/org/apache/kafka/storage/internals/log/TimeIndex.java: ########## @@ -74,13 +74,13 @@ public void sanityCheck() { TimestampOffset entry = lastEntry(); long lastTimestamp = entry.timestamp; long lastOffset = entry.offset; + if (entries() != 0 && lastOffset < baseOffset()) + throw new CorruptIndexException("Corrupt time index found, time index file (" + file().getAbsolutePath() + ") has " + + "non-zero size but the last offset is " + lastOffset + " which is less than the first offset " + baseOffset()); Review Comment: This is a nice change. We should fail fast by checking local state even before calling the expensive mmap() function below. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org