diff --git a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala index 30caec1..da471cd 100644 --- a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala +++ b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala @@ -64,8 +64,12 @@ class HighwaterMarkCheckpoint(val path: String) extends Logging { hwFileWriter.close() // swap new high watermark file with previous one if(!tempHwFile.renameTo(hwFile)) { - fatal("Attempt to swap the new high watermark file with the old one failed") - System.exit(1) + // renameTo() fails on Windows if the destination file exists. + hwFile.delete() + if(!tempHwFile.renameTo(hwFile)) { + fatal("Attempt to swap the new high watermark file with the old one failed") + System.exit(1) + } } }finally { hwFileLock.unlock()