You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jk...@apache.org on 2012/12/18 18:44:12 UTC

[14/30] git commit: KAFKA-654 Irrecoverable error while trying to roll a segment that already exists; patched by Neha Narkhede; reviewed by Jay Kreps

KAFKA-654 Irrecoverable error while trying to roll a segment that already exists; patched by Neha Narkhede; reviewed by Jay Kreps

git-svn-id: https://svn.apache.org/repos/asf/kafka/branches/0.8@1419627 13f79535-47bb-0310-9956-ffa450edef68


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

Branch: refs/heads/trunk
Commit: 133c88278da2469bdf5c3d012a03c5eb117022bd
Parents: 8a93698
Author: Neha Narkhede <ne...@apache.org>
Authored: Mon Dec 10 18:23:22 2012 +0000
Committer: Neha Narkhede <ne...@apache.org>
Committed: Mon Dec 10 18:23:22 2012 +0000

----------------------------------------------------------------------
 core/src/main/scala/kafka/log/Log.scala |    6 +++---
 1 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/133c8827/core/src/main/scala/kafka/log/Log.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index c2fccec..96bf2ed 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -591,12 +591,12 @@ private[kafka] class Log(val dir: File,
   def truncateAndStartWithNewOffset(newOffset: Long) {
     lock synchronized {
       val deletedSegments = segments.trunc(segments.view.size)
-      debug("Truncate and start log '" + name + "' to " + newOffset)
-      segments.append(new LogSegment(dir, 
+      info("Truncate and start log '" + name + "' to " + newOffset)
+      deleteSegments(deletedSegments)
+      segments.append(new LogSegment(dir,
                                      newOffset,
                                      indexIntervalBytes = indexIntervalBytes, 
                                      maxIndexSize = maxIndexSize))
-      deleteSegments(deletedSegments)
       this.nextOffset.set(newOffset)
     }
   }