/
Log.scala
664 lines (588 loc) · 25.2 KB
/
Log.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log
import kafka.api.OffsetRequest
import java.io.{IOException, File}
import java.util.{Comparator, Collections, ArrayList}
import java.util.concurrent.atomic._
import kafka.utils._
import scala.math._
import java.text.NumberFormat
import kafka.server.BrokerTopicStats
import kafka.message._
import kafka.common._
import kafka.metrics.KafkaMetricsGroup
import com.yammer.metrics.core.Gauge
object Log {
val LogFileSuffix = ".log"
val IndexFileSuffix = ".index"
/**
* Search for the greatest range with start <= the target value.
*/
def findRange[T <: Range](ranges: Array[T], value: Long, arraySize: Int): Option[T] = {
if(ranges.size < 1)
return None
// check out of bounds
if(value < ranges(0).start)
return None
var low = 0
var high = arraySize - 1
while(low < high) {
val mid = ceil((high + low) / 2.0).toInt
val found = ranges(mid)
if(found.start == value)
return Some(found)
else if (value < found.start)
high = mid - 1
else
low = mid
}
Some(ranges(low))
}
def findRange[T <: Range](ranges: Array[T], value: Long): Option[T] =
findRange(ranges, value, ranges.length)
/**
* Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
* so that ls sorts the files numerically
*/
def filenamePrefixFromOffset(offset: Long): String = {
val nf = NumberFormat.getInstance()
nf.setMinimumIntegerDigits(20)
nf.setMaximumFractionDigits(0)
nf.setGroupingUsed(false)
nf.format(offset)
}
def logFilename(dir: File, offset: Long) =
new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix)
def indexFilename(dir: File, offset: Long) =
new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix)
def getEmptyOffsets(timestamp: Long): Seq[Long] =
if (timestamp == OffsetRequest.LatestTime || timestamp == OffsetRequest.EarliestTime)
Seq(0L)
else Nil
}
/**
* An append-only log for storing messages.
*
* The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
*
* New log segments are created according to a configurable policy that controls the size in bytes or time interval
* for a given segment.
*
*/
@threadsafe
private[kafka] class Log(val dir: File,
val maxLogFileSize: Int,
val maxMessageSize: Int,
val flushInterval: Int = Int.MaxValue,
val rollIntervalMs: Long = Long.MaxValue,
val needsRecovery: Boolean,
val maxIndexSize: Int = (10*1024*1024),
val indexIntervalBytes: Int = 4096,
time: Time = SystemTime,
brokerId: Int = 0) extends Logging with KafkaMetricsGroup {
this.logIdent = "[Kafka Log on Broker " + brokerId + "], "
import kafka.log.Log._
/* A lock that guards all modifications to the log */
private val lock = new Object
/* The current number of unflushed messages appended to the write */
private val unflushed = new AtomicInteger(0)
/* last time it was flushed */
private val lastflushedTime = new AtomicLong(System.currentTimeMillis)
/* the actual segments of the log */
private[log] val segments: SegmentList[LogSegment] = loadSegments()
/* Calculate the offset of the next message */
private var nextOffset: AtomicLong = new AtomicLong(segments.view.last.nextOffset())
debug("Completed load of log %s with log end offset %d".format(name, logEndOffset))
newGauge(name + "-" + "NumLogSegments",
new Gauge[Int] { def getValue = numberOfSegments })
newGauge(name + "-" + "LogEndOffset",
new Gauge[Long] { def getValue = logEndOffset })
/* The name of this log */
def name = dir.getName()
/* Load the log segments from the log files on disk */
private def loadSegments(): SegmentList[LogSegment] = {
// open all the segments read-only
val logSegments = new ArrayList[LogSegment]
val ls = dir.listFiles()
if(ls != null) {
for(file <- ls if file.isFile) {
val filename = file.getName()
if(!file.canRead) {
throw new IOException("Could not read file " + file)
} else if(filename.endsWith(IndexFileSuffix)) {
// ensure that we have a corresponding log file for this index file
val log = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix))
if(!log.exists) {
warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath))
file.delete()
}
} else if(filename.endsWith(LogFileSuffix)) {
val offset = filename.substring(0, filename.length - LogFileSuffix.length).toLong
// TODO: we should ideally rebuild any missing index files, instead of erroring out
if(!Log.indexFilename(dir, offset).exists)
throw new IllegalStateException("Found log file with no corresponding index file.")
logSegments.add(new LogSegment(dir = dir,
startOffset = offset,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize))
}
}
}
if(logSegments.size == 0) {
// no existing segments, create a new mutable segment
logSegments.add(new LogSegment(dir = dir,
startOffset = 0,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize))
} else {
// there is at least one existing segment, validate and recover them/it
// sort segments into ascending order for fast searching
Collections.sort(logSegments, new Comparator[LogSegment] {
def compare(s1: LogSegment, s2: LogSegment): Int = {
if(s1.start == s2.start) 0
else if(s1.start < s2.start) -1
else 1
}
})
// reset the index size of the last (current active) log segment to its maximum value
logSegments.get(logSegments.size() - 1).index.resize(maxIndexSize)
// run recovery on the last segment if necessary
if(needsRecovery)
recoverSegment(logSegments.get(logSegments.size - 1))
}
val segmentList = logSegments.toArray(new Array[LogSegment](logSegments.size))
// Check for the index file of every segment, if it's empty or its last offset is greater than its base offset.
for (s <- segmentList) {
require(s.index.entries == 0 || s.index.lastOffset > s.index.baseOffset,
"Corrupt index found, index file (%s) has non-zero size but the last offset is %d and the base offset is %d"
.format(s.index.file.getAbsolutePath, s.index.lastOffset, s.index.baseOffset))
}
new SegmentList(segmentList)
}
/**
* Run recovery on the given segment. This will rebuild the index from the log file and lop off any invalid bytes from the end of the log.
*/
private def recoverSegment(segment: LogSegment) {
info("Recovering log segment %s".format(segment.messageSet.file.getAbsolutePath))
segment.index.truncate()
var validBytes = 0
var lastIndexEntry = 0
val iter = segment.messageSet.iterator
try {
while(iter.hasNext) {
val entry = iter.next
entry.message.ensureValid()
if(validBytes - lastIndexEntry > indexIntervalBytes) {
segment.index.append(entry.offset, validBytes)
lastIndexEntry = validBytes
}
validBytes += MessageSet.entrySize(entry.message)
}
} catch {
case e: InvalidMessageException =>
logger.warn("Found invalid messages in log " + name)
}
val truncated = segment.messageSet.sizeInBytes - validBytes
if(truncated > 0)
warn("Truncated " + truncated + " invalid bytes from the log " + name + ".")
segment.messageSet.truncateTo(validBytes)
}
/**
* The number of segments in the log
*/
def numberOfSegments: Int = segments.view.length
/**
* Close this log
*/
def close() {
debug("Closing log " + name)
lock synchronized {
for(seg <- segments.view)
seg.close()
}
}
/**
* Append this message set to the active segment of the log, rolling over to a fresh segment if necessary.
*
* This method will generally be responsible for assigning offsets to the messages,
* however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
*
* Returns a tuple containing (first_offset, last_offset) for the newly appended of the message set,
* or (-1,-1) if the message set is empty
*/
def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): (Long, Long) = {
val messageSetInfo = analyzeAndValidateMessageSet(messages)
// if we have any valid messages, append them to the log
if(messageSetInfo.count == 0) {
(-1L, -1L)
} else {
// trim any invalid bytes or partial messages before appending it to the on-disk log
var validMessages = trimInvalidBytes(messages)
try {
// they are valid, insert them in the log
val offsets = lock synchronized {
// maybe roll the log if this segment is full
val segment = maybeRoll(segments.view.last)
// assign offsets to the messageset
val offsets =
if(assignOffsets) {
val offsetCounter = new AtomicLong(nextOffset.get)
val firstOffset = offsetCounter.get
try {
validMessages = validMessages.assignOffsets(offsetCounter, messageSetInfo.codec)
} catch {
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
}
val lastOffset = offsetCounter.get - 1
val numMessages = lastOffset - firstOffset + 1
BrokerTopicStats.getBrokerTopicStats(topicName).messagesInRate.mark(numMessages)
BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numMessages)
(firstOffset, lastOffset)
} else {
require(messageSetInfo.offsetsMonotonic, "Out of order offsets found in " + messages)
require(messageSetInfo.firstOffset >= nextOffset.get,
"Attempt to append a message set beginning with offset %d to a log with log end offset %d."
.format(messageSetInfo.firstOffset, nextOffset.get))
(messageSetInfo.firstOffset, messageSetInfo.lastOffset)
}
// Check if the message sizes are valid. This check is done after assigning offsets to ensure the comparison
// happens with the new message size (after re-compression, if any)
for(messageAndOffset <- validMessages.shallowIterator) {
if(MessageSet.entrySize(messageAndOffset.message) > maxMessageSize)
throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
.format(MessageSet.entrySize(messageAndOffset.message), maxMessageSize))
}
// now append to the log
trace("Appending message set to %s offset: %d nextOffset: %d messageSet: %s"
.format(this.name, offsets._1, nextOffset.get(), validMessages))
segment.append(offsets._1, validMessages)
// advance the log end offset
nextOffset.set(offsets._2 + 1)
// return the offset at which the messages were appended
offsets
}
// maybe flush the log and index
maybeFlush(messageSetInfo.count)
// return the first and last offset
offsets
} catch {
case e: IOException => throw new KafkaStorageException("I/O exception in append to log '%s'".format(name), e)
}
}
}
/* struct to hold various quantities we compute about each message set before appending to the log */
case class MessageSetAppendInfo(firstOffset: Long, lastOffset: Long, codec: CompressionCodec, count: Int, offsetsMonotonic: Boolean)
/**
* Validate the following:
* 1. each message matches its CRC
*
* Also compute the following quantities:
* 1. First offset in the message set
* 2. Last offset in the message set
* 3. Number of messages
* 4. Whether the offsets are monotonically increasing
* 5. Whether any compression codec is used (if many are used, then the last one is given)
*/
private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): MessageSetAppendInfo = {
var messageCount = 0
var firstOffset, lastOffset = -1L
var codec: CompressionCodec = NoCompressionCodec
var monotonic = true
for(messageAndOffset <- messages.shallowIterator) {
// update the first offset if on the first message
if(firstOffset < 0)
firstOffset = messageAndOffset.offset
// check that offsets are monotonically increasing
if(lastOffset >= messageAndOffset.offset)
monotonic = false
// update the last offset seen
lastOffset = messageAndOffset.offset
// check the validity of the message by checking CRC
val m = messageAndOffset.message
m.ensureValid()
messageCount += 1;
val messageCodec = m.compressionCodec
if(messageCodec != NoCompressionCodec)
codec = messageCodec
}
MessageSetAppendInfo(firstOffset, lastOffset, codec, messageCount, monotonic)
}
/**
* Trim any invalid bytes from the end of this message set (if there are any)
*/
private def trimInvalidBytes(messages: ByteBufferMessageSet): ByteBufferMessageSet = {
val messageSetValidBytes = messages.validBytes
if(messageSetValidBytes < 0)
throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
if(messageSetValidBytes == messages.sizeInBytes) {
messages
} else {
// trim invalid bytes
val validByteBuffer = messages.buffer.duplicate()
validByteBuffer.limit(messageSetValidBytes)
new ByteBufferMessageSet(validByteBuffer)
}
}
/**
* Read a message set from the log.
* startOffset - The logical offset to begin reading at
* maxLength - The maximum number of bytes to read
* maxOffset - The first offset not included in the read
*/
def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None): MessageSet = {
trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size))
val view = segments.view
// check if the offset is valid and in range
val first = view.head.start
val next = nextOffset.get
if(startOffset == next)
return MessageSet.Empty
else if(startOffset > next || startOffset < first)
throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, first, next))
// Do the read on the segment with a base offset less than the target offset
// TODO: to handle sparse offsets, we need to skip to the next segment if this read doesn't find anything
Log.findRange(view, startOffset, view.length) match {
case None => throw new OffsetOutOfRangeException("Offset is earlier than the earliest offset")
case Some(segment) => segment.read(startOffset, maxLength, maxOffset)
}
}
/**
* Delete any log segments matching the given predicate function
*/
def markDeletedWhile(predicate: LogSegment => Boolean): Seq[LogSegment] = {
lock synchronized {
debug("Garbage collecting log..")
debug("Segments of log %s : %s ".format(this.name, segments.view.mkString(",")))
debug("Index files for log %s: %s".format(this.name, segments.view.map(_.index.file.exists()).mkString(",")))
debug("Data files for log %s: %s".format(this.name, segments.view.map(_.messageSet.file.exists()).mkString(",")))
val view = segments.view
val deletable = view.takeWhile(predicate)
for(seg <- deletable)
seg.deleted = true
var numToDelete = deletable.size
// if we are deleting everything, create a new empty segment
if(numToDelete == view.size) {
if (view(numToDelete - 1).size > 0)
roll()
else {
// If the last segment to be deleted is empty and we roll the log, the new segment will have the same
// file name. So simply reuse the last segment and reset the modified time.
view(numToDelete - 1).messageSet.file.setLastModified(time.milliseconds)
numToDelete -=1
}
}
segments.trunc(numToDelete)
}
}
/**
* Get the size of the log in bytes
*/
def size: Long = segments.view.foldLeft(0L)(_ + _.size)
/**
* Get the offset of the next message that will be appended
*/
def logEndOffset: Long = nextOffset.get
/**
* Roll the log over if necessary
*/
private def maybeRoll(segment: LogSegment): LogSegment = {
if(segment.messageSet.sizeInBytes > maxLogFileSize) {
info("Rolling %s due to full data log".format(name))
roll()
} else if((segment.firstAppendTime.isDefined) && (time.milliseconds - segment.firstAppendTime.get > rollIntervalMs)) {
info("Rolling %s due to time based rolling".format(name))
roll()
} else if(segment.index.isFull) {
info("Rolling %s due to full index maxIndexSize = %d, entries = %d, maxEntries = %d"
.format(name, segment.index.maxIndexSize, segment.index.entries(), segment.index.maxEntries))
roll()
} else
segment
}
/**
* Create a new segment and make it active, and return it
*/
def roll(): LogSegment = {
lock synchronized {
flush()
rollToOffset(logEndOffset)
}
}
/**
* Roll the log over to the given new offset value
*/
private def rollToOffset(newOffset: Long): LogSegment = {
val logFile = logFilename(dir, newOffset)
val indexFile = indexFilename(dir, newOffset)
for(file <- List(logFile, indexFile); if file.exists) {
warn("Newly rolled segment file " + file.getAbsolutePath + " already exists; deleting it first")
file.delete()
}
info("Rolling log '" + name + "' to " + logFile.getAbsolutePath + " and " + indexFile.getAbsolutePath)
segments.view.lastOption match {
case Some(segment) => segment.index.trimToValidSize()
case None =>
}
val segmentsView = segments.view
if(segmentsView.size > 0 && segmentsView.last.start == newOffset)
throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists".format(dir.getName, newOffset))
val segment = new LogSegment(dir,
startOffset = newOffset,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize)
segments.append(segment)
segment
}
/**
* Flush the log if necessary
*/
private def maybeFlush(numberOfMessages : Int) {
if(unflushed.addAndGet(numberOfMessages) >= flushInterval)
flush()
}
/**
* Flush this log file to the physical disk
*/
def flush() : Unit = {
if (unflushed.get == 0)
return
lock synchronized {
debug("Flushing log '" + name + "' last flushed: " + getLastFlushedTime + " current time: " +
time.milliseconds)
segments.view.last.flush()
unflushed.set(0)
lastflushedTime.set(time.milliseconds)
}
}
def getOffsetsBefore(timestamp: Long, maxNumOffsets: Int): Seq[Long] = {
val segsArray = segments.view
var offsetTimeArray: Array[(Long, Long)] = null
if(segsArray.last.size > 0)
offsetTimeArray = new Array[(Long, Long)](segsArray.length + 1)
else
offsetTimeArray = new Array[(Long, Long)](segsArray.length)
for(i <- 0 until segsArray.length)
offsetTimeArray(i) = (segsArray(i).start, segsArray(i).messageSet.file.lastModified)
if(segsArray.last.size > 0)
offsetTimeArray(segsArray.length) = (logEndOffset, time.milliseconds)
var startIndex = -1
timestamp match {
case OffsetRequest.LatestTime =>
startIndex = offsetTimeArray.length - 1
case OffsetRequest.EarliestTime =>
startIndex = 0
case _ =>
var isFound = false
debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2)))
startIndex = offsetTimeArray.length - 1
while (startIndex >= 0 && !isFound) {
if (offsetTimeArray(startIndex)._2 <= timestamp)
isFound = true
else
startIndex -=1
}
}
val retSize = maxNumOffsets.min(startIndex + 1)
val ret = new Array[Long](retSize)
for(j <- 0 until retSize) {
ret(j) = offsetTimeArray(startIndex)._1
startIndex -= 1
}
// ensure that the returned seq is in descending order of offsets
ret.toSeq.sortBy(- _)
}
def delete(): Unit = {
deleteSegments(segments.contents.get())
Utils.rm(dir)
}
/* Attempts to delete all provided segments from a log and returns how many it was able to */
def deleteSegments(segments: Seq[LogSegment]): Int = {
var total = 0
for(segment <- segments) {
info("Deleting log segment " + segment.start + " from " + name)
val deletedLog = segment.messageSet.delete()
val deletedIndex = segment.index.delete()
if(!deletedIndex || !deletedLog) {
throw new KafkaStorageException("Deleting log segment " + segment.start + " failed.")
} else {
total += 1
}
if(segment.messageSet.file.exists())
error("Data log file %s still exists".format(segment.messageSet.file.getAbsolutePath))
if(segment.index.file.exists())
error("Index file %s still exists".format(segment.index.file.getAbsolutePath))
}
total
}
def truncateTo(targetOffset: Long) {
if(targetOffset < 0)
throw new IllegalArgumentException("Cannot truncate to a negative offset (%d).".format(targetOffset))
lock synchronized {
val segmentsToBeDeleted = segments.view.filter(segment => segment.start > targetOffset)
val viewSize = segments.view.size
val numSegmentsDeleted = deleteSegments(segmentsToBeDeleted)
/* We should not hit this error because segments.view is locked in markedDeletedWhile() */
if(numSegmentsDeleted != segmentsToBeDeleted.size)
error("Failed to delete some segments when attempting to truncate to offset " + targetOffset +")")
if (numSegmentsDeleted == viewSize) {
segments.trunc(segments.view.size)
rollToOffset(targetOffset)
this.nextOffset.set(targetOffset)
} else {
if(targetOffset > logEndOffset) {
error("Target offset %d cannot be greater than the last message offset %d in the log %s".
format(targetOffset, logEndOffset, segments.view.last.messageSet.file.getAbsolutePath))
} else {
// find the log segment that has this hw
val segmentToBeTruncated = findRange(segments.view, targetOffset)
segmentToBeTruncated match {
case Some(segment) =>
val truncatedSegmentIndex = segments.view.indexOf(segment)
segments.truncLast(truncatedSegmentIndex)
segment.truncateTo(targetOffset)
this.nextOffset.set(targetOffset)
info("Truncated log segment %s to target offset %d".format(segments.view.last.messageSet.file.getAbsolutePath, targetOffset))
case None => // nothing to do
}
}
}
}
}
/**
* Truncate all segments in the log and start a new segment on a new offset
*/
def truncateAndStartWithNewOffset(newOffset: Long) {
lock synchronized {
val deletedSegments = segments.trunc(segments.view.size)
info("Truncate and start log '" + name + "' to " + newOffset)
deleteSegments(deletedSegments)
segments.append(new LogSegment(dir,
newOffset,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize))
this.nextOffset.set(newOffset)
}
}
def topicName():String = {
name.substring(0, name.lastIndexOf("-"))
}
def getLastFlushedTime():Long = {
return lastflushedTime.get
}
override def toString() = "Log(" + this.dir + ")"
}