jsancio commented on a change in pull request #10021:
URL: https://github.com/apache/kafka/pull/10021#discussion_r577852672



##########
File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
##########
@@ -267,30 +259,45 @@ final class KafkaMetadataLog private (
     }
   }
 
-  override def oldestSnapshotId(): Optional[OffsetAndEpoch] = {
-    oldestSnapshotId
+  override def earliestSnapshotId(): Optional[OffsetAndEpoch] = {
+    try {
+      Optional.of(snapshotIds.first)
+    } catch {
+      case _: NoSuchElementException =>
+        Optional.empty()
+    }
   }
 
   override def onSnapshotFrozen(snapshotId: OffsetAndEpoch): Unit = {
     snapshotIds.add(snapshotId)
   }
 
   override def deleteBeforeSnapshot(logStartSnapshotId: OffsetAndEpoch): 
Boolean = {
-    latestSnapshotId.asScala match {
+    latestSnapshotId().asScala match {
       case Some(snapshotId) if (snapshotIds.contains(logStartSnapshotId) &&
         startOffset < logStartSnapshotId.offset &&
         logStartSnapshotId.offset <= snapshotId.offset &&
         log.maybeIncrementLogStartOffset(logStartSnapshotId.offset, 
SnapshotGenerated)) =>
-
         log.deleteOldSegments()
-        oldestSnapshotId = Optional.of(logStartSnapshotId)
+
+        // Delete snapshot after increasing LogStartOffset
+        removeSnapshotFilesBefore(logStartSnapshotId)
 
         true
 
       case _ => false
     }
   }
 
+  /**
+   * Removes all snapshots on the log directory whose epoch and end offset is 
less than the giving epoch and end offset.
+   */
+  private def removeSnapshotFilesBefore(logStartSnapshotId: OffsetAndEpoch): 
Unit = {
+    val expiredSnapshotIds = snapshotIds.headSet(logStartSnapshotId)
+    expiredSnapshotIds.forEach(snapshotId => 
Snapshots.deleteSnapshotIfExists(log.dir.toPath, snapshotId))

Review comment:
       One of the invariant for this data structure has is that if 
`snaspshotIds` contains a snapshot id the `KafkaRaftClient` and `Listener` can 
expect that the snapshot exists on the file system. With this implementation 
this is not strictly the case since we are first deleting the snapshot from the 
file system and then we are removing it from `snapshotIds`. What if we do the 
opposite? E.g.
   
   ```scala
   import Breaks.{breakable, break}
   
   breakable {
     val iterator = snapshotIds.iterator()
     while (iterator.hasNext()) {
       val snapshotId = iterator.next()
       if (snapshotId < logStartSnapshotId) {
         iterator.remove()
         Snapshots.deleteSnapshotIfExists(log.dir.toPath, snapshotId)
       } else {
         break
       }
     }
   }
   ```




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to