Index: core/src/main/scala/kafka/log/Log.scala =================================================================== --- core/src/main/scala/kafka/log/Log.scala (revision 1419108) +++ core/src/main/scala/kafka/log/Log.scala (working copy) @@ -591,12 +591,12 @@ 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) } }