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


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/43c43b1c
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/43c43b1c
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/43c43b1c

Branch: refs/heads/trunk
Commit: 43c43b1c6d05fb93f4ca96a4e1a1aa8c6fd58d18
Parents: 4f387ae
Author: Jun Rao <jun...@gmail.com>
Authored: Mon Jun 3 16:51:41 2013 -0700
Committer: Jun Rao <jun...@gmail.com>
Committed: Mon Jun 3 16:51:41 2013 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/server/HighwaterMarkCheckpoint.scala | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/43c43b1c/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala 
b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala
index 30caec1..e18ecab 100644
--- a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala
+++ b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala
@@ -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()

Reply via email to