kafka-903; Attempt to swap the new high watermark file with the old one failed on Windows; patched by Jun Rao; reviewed by Neha Narkhede, Jay Kreps and Sriram Subramania

This commit is contained in:
Jun Rao 2013-06-03 16:51:41 -07:00
parent 4f387ae435
commit 43c43b1c6d
1 changed files with 7 additions and 3 deletions

View File

@ -36,7 +36,7 @@ object HighwaterMarkCheckpoint {
class HighwaterMarkCheckpoint(val path: String) extends Logging {
/* create the highwatermark file handle for all partitions */
val name = path + "/" + HighwaterMarkCheckpoint.highWatermarkFileName
val name = path + File.separator + HighwaterMarkCheckpoint.highWatermarkFileName
private val hwFile = new File(name)
private val hwFileLock = new ReentrantLock()
// recover from previous tmp file, if required
@ -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()