forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
LogLoaderTest.scala
1809 lines (1599 loc) · 87.8 KB
/
LogLoaderTest.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
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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 java.io.{BufferedWriter, File, FileWriter, IOException}
import java.nio.ByteBuffer
import java.nio.file.{Files, NoSuchFileException, Paths}
import java.util.{Optional, OptionalLong, Properties}
import kafka.server.{BrokerTopicStats, KafkaConfig}
import kafka.server.metadata.MockConfigRepository
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType}
import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0
import org.apache.kafka.server.util.{MockTime, Scheduler}
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile}
import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
import org.junit.jupiter.api.function.Executable
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.CsvSource
import org.mockito.{ArgumentMatchers, Mockito}
import org.mockito.ArgumentMatchers.{any, anyLong}
import org.mockito.Mockito.{mock, reset, times, verify, when}
import java.util.concurrent.ConcurrentMap
import scala.annotation.nowarn
import scala.collection.mutable.ListBuffer
import scala.collection.{Iterable, Map, mutable}
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
class LogLoaderTest {
var config: KafkaConfig = _
val brokerTopicStats = new BrokerTopicStats
val maxTransactionTimeoutMs: Int = 5 * 60 * 1000
val producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false)
val producerIdExpirationCheckIntervalMs: Int = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT
val tmpDir = TestUtils.tempDir()
val logDir = TestUtils.randomPartitionLogDir(tmpDir)
var logsToClose: Seq[UnifiedLog] = Seq()
val mockTime = new MockTime()
@BeforeEach
def setUp(): Unit = {
val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
config = KafkaConfig.fromProps(props)
}
@AfterEach
def tearDown(): Unit = {
brokerTopicStats.close()
logsToClose.foreach(l => Utils.closeQuietly(l, "UnifiedLog"))
Utils.delete(tmpDir)
}
object ErrorTypes extends Enumeration {
type Errors = Value
val IOException, RuntimeException, KafkaStorageExceptionWithIOExceptionCause,
KafkaStorageExceptionWithoutIOExceptionCause = Value
}
@Test
def testLogRecoveryIsCalledUponBrokerCrash(): Unit = {
// LogManager must realize correctly if the last shutdown was not clean and the logs need
// to run recovery while loading upon subsequent broker boot up.
val logDir: File = TestUtils.tempDir()
val logProps = new Properties()
val logConfig = new LogConfig(logProps)
val logDirs = Seq(logDir)
val topicPartition = new TopicPartition("foo", 0)
var log: UnifiedLog = null
val time = new MockTime()
var cleanShutdownInterceptedValue = false
case class SimulateError(var hasError: Boolean = false, var errorType: ErrorTypes.Errors = ErrorTypes.RuntimeException)
val simulateError = SimulateError()
val logDirFailureChannel = new LogDirFailureChannel(logDirs.size)
val maxTransactionTimeoutMs = 5 * 60 * 1000
val producerIdExpirationCheckIntervalMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT
// Create a LogManager with some overridden methods to facilitate interception of clean shutdown
// flag and to inject an error
def interceptedLogManager(logConfig: LogConfig,
logDirs: Seq[File],
logDirFailureChannel: LogDirFailureChannel
): LogManager = {
new LogManager(
logDirs = logDirs.map(_.getAbsoluteFile),
initialOfflineDirs = Array.empty[File],
configRepository = new MockConfigRepository(),
initialDefaultConfig = logConfig,
cleanerConfig = new CleanerConfig(false),
recoveryThreadsPerDataDir = 4,
flushCheckMs = 1000L,
flushRecoveryOffsetCheckpointMs = 10000L,
flushStartOffsetCheckpointMs = 10000L,
retentionCheckMs = 1000L,
maxTransactionTimeoutMs = maxTransactionTimeoutMs,
producerStateManagerConfig = producerStateManagerConfig,
producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs,
interBrokerProtocolVersion = config.interBrokerProtocolVersion,
scheduler = time.scheduler,
brokerTopicStats = new BrokerTopicStats(),
logDirFailureChannel = logDirFailureChannel,
time = time,
keepPartitionMetadataFile = config.usesTopicId,
remoteStorageSystemEnable = config.remoteLogManagerConfig.enableRemoteStorageSystem(),
initialTaskDelayMs = config.logInitialTaskDelayMs) {
override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long],
logStartOffsets: Map[TopicPartition, Long], defaultConfig: LogConfig,
topicConfigs: Map[String, LogConfig], numRemainingSegments: ConcurrentMap[String, Int],
shouldBeStrayKraftLog: UnifiedLog => Boolean): UnifiedLog = {
if (simulateError.hasError) {
simulateError.errorType match {
case ErrorTypes.KafkaStorageExceptionWithIOExceptionCause =>
throw new KafkaStorageException(new IOException("Simulated Kafka storage error with IOException cause"))
case ErrorTypes.KafkaStorageExceptionWithoutIOExceptionCause =>
throw new KafkaStorageException("Simulated Kafka storage error without IOException cause")
case ErrorTypes.IOException =>
throw new IOException("Simulated IO error")
case _ =>
throw new RuntimeException("Simulated Runtime error")
}
}
cleanShutdownInterceptedValue = hadCleanShutdown
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L)
val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1)
val segments = new LogSegments(topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val producerStateManager = new ProducerStateManager(topicPartition, logDir,
this.maxTransactionTimeoutMs, this.producerStateManagerConfig, time)
val logLoader = new LogLoader(logDir, topicPartition, config, time.scheduler, time,
logDirFailureChannel, hadCleanShutdown, segments, logStartOffset, logRecoveryPoint,
leaderEpochCache.asJava, producerStateManager)
val offsets = logLoader.load()
val localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition,
logDirFailureChannel)
new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats,
this.producerIdExpirationCheckIntervalMs, leaderEpochCache,
producerStateManager, None, true)
}
}
}
def initializeLogManagerForSimulatingErrorTest(logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(logDirs.size)
): (LogManager, Executable) = {
val logManager: LogManager = interceptedLogManager(logConfig, logDirs, logDirFailureChannel)
log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None)
assertFalse(logDirFailureChannel.hasOfflineLogDir(logDir.getAbsolutePath), "log dir should not be offline before load logs")
val runLoadLogs: Executable = () => {
val defaultConfig = logManager.currentDefaultConfig
logManager.loadLogs(defaultConfig, logManager.fetchTopicConfigOverrides(defaultConfig, Set.empty), _ => false)
}
(logManager, runLoadLogs)
}
val cleanShutdownFileHandler = new CleanShutdownFileHandler(logDir.getPath)
locally {
val (logManager, _) = initializeLogManagerForSimulatingErrorTest()
// Load logs after a clean shutdown
cleanShutdownFileHandler.write(0L)
cleanShutdownInterceptedValue = false
var defaultConfig = logManager.currentDefaultConfig
logManager.loadLogs(defaultConfig, logManager.fetchTopicConfigOverrides(defaultConfig, Set.empty), _ => false)
assertTrue(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag")
assertFalse(cleanShutdownFileHandler.exists(), "Clean shutdown file must not exist after loadLogs has completed")
// Load logs without clean shutdown file
cleanShutdownInterceptedValue = true
defaultConfig = logManager.currentDefaultConfig
logManager.loadLogs(defaultConfig, logManager.fetchTopicConfigOverrides(defaultConfig, Set.empty), _ => false)
assertFalse(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag")
assertFalse(cleanShutdownFileHandler.exists(), "Clean shutdown file must not exist after loadLogs has completed")
// Create clean shutdown file and then simulate error while loading logs such that log loading does not complete.
cleanShutdownFileHandler.write(0L)
logManager.shutdown()
}
locally {
val (logManager, runLoadLogs) = initializeLogManagerForSimulatingErrorTest(logDirFailureChannel)
// Simulate Runtime error
simulateError.hasError = true
simulateError.errorType = ErrorTypes.RuntimeException
assertThrows(classOf[RuntimeException], runLoadLogs)
assertFalse(cleanShutdownFileHandler.exists(), "Clean shutdown file must not have existed")
assertFalse(logDirFailureChannel.hasOfflineLogDir(logDir.getAbsolutePath), "log dir should not turn offline when Runtime Exception thrown")
// Simulate Kafka storage error with IOException cause
// in this case, the logDir will be added into offline list before KafkaStorageThrown. So we don't verify it here
simulateError.errorType = ErrorTypes.KafkaStorageExceptionWithIOExceptionCause
assertDoesNotThrow(runLoadLogs, "KafkaStorageException with IOException cause should be caught and handled")
// Simulate Kafka storage error without IOException cause
simulateError.errorType = ErrorTypes.KafkaStorageExceptionWithoutIOExceptionCause
assertThrows(classOf[KafkaStorageException], runLoadLogs, "should throw exception when KafkaStorageException without IOException cause")
assertFalse(logDirFailureChannel.hasOfflineLogDir(logDir.getAbsolutePath), "log dir should not turn offline when KafkaStorageException without IOException cause thrown")
// Simulate IO error
simulateError.errorType = ErrorTypes.IOException
assertDoesNotThrow(runLoadLogs, "IOException should be caught and handled")
assertTrue(logDirFailureChannel.hasOfflineLogDir(logDir.getAbsolutePath), "the log dir should turn offline after IOException thrown")
// Do not simulate error on next call to LogManager#loadLogs. LogManager must understand that log had unclean shutdown the last time.
simulateError.hasError = false
cleanShutdownInterceptedValue = true
val defaultConfig = logManager.currentDefaultConfig
logManager.loadLogs(defaultConfig, logManager.fetchTopicConfigOverrides(defaultConfig, Set.empty), _ => false)
assertFalse(cleanShutdownInterceptedValue, "Unexpected value for clean shutdown flag")
logManager.shutdown()
}
}
@Test
def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = {
testProducerSnapshotsRecoveryAfterUncleanShutdown(MetadataVersion.minSupportedFor(RecordVersion.V1).version)
}
@Test
def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = {
testProducerSnapshotsRecoveryAfterUncleanShutdown(MetadataVersion.latestTesting.version)
}
private def createLog(dir: File,
config: LogConfig,
brokerTopicStats: BrokerTopicStats = brokerTopicStats,
logStartOffset: Long = 0L,
recoveryPoint: Long = 0L,
scheduler: Scheduler = mockTime.scheduler,
time: Time = mockTime,
maxTransactionTimeoutMs: Int = maxTransactionTimeoutMs,
maxProducerIdExpirationMs: Int = producerStateManagerConfig.producerIdExpirationMs,
producerIdExpirationCheckIntervalMs: Int = producerIdExpirationCheckIntervalMs,
lastShutdownClean: Boolean = true): UnifiedLog = {
val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint,
maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, false), producerIdExpirationCheckIntervalMs, lastShutdownClean)
logsToClose = logsToClose :+ log
log
}
private def createLogWithOffsetOverflow(logConfig: LogConfig): (UnifiedLog, LogSegment) = {
LogTestUtils.initializeLogDirWithOverflowedSegment(logDir)
val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue)
val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse {
throw new AssertionError("Failed to create log with a segment which has overflowed offsets")
}
(log, segmentWithOverflow)
}
private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): UnifiedLog = {
// method is called only in case of recovery from hard reset
val recoveredLog = LogTestUtils.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler)
logsToClose = logsToClose :+ recoveredLog
recoveredLog
}
/**
* Wrap a single record log buffer with leader epoch.
*/
private def singletonRecordsWithLeaderEpoch(value: Array[Byte],
key: Array[Byte] = null,
leaderEpoch: Int,
offset: Long,
codec: CompressionType = CompressionType.NONE,
timestamp: Long = RecordBatch.NO_TIMESTAMP,
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
val records = Seq(new SimpleRecord(timestamp, key, value))
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset,
mockTime.milliseconds, leaderEpoch)
records.foreach(builder.append)
builder.build()
}
@nowarn("cat=deprecation")
private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640")
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, messageFormatVersion)
val logConfig = new LogConfig(logProps)
var log = createLog(logDir, logConfig)
assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset)
for (i <- 0 to 100) {
val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes)
log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0)
}
assertTrue(log.logSegments.size >= 5)
val segmentOffsets = log.logSegments.asScala.toVector.map(_.baseOffset)
val activeSegmentOffset = segmentOffsets.last
// We want the recovery point to be past the segment offset and before the last 2 segments including a gap of
// 1 segment. We collect the data before closing the log.
val offsetForSegmentAfterRecoveryPoint = segmentOffsets(segmentOffsets.size - 3)
val offsetForRecoveryPointSegment = segmentOffsets(segmentOffsets.size - 4)
val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.toSet.partition(_ < offsetForRecoveryPointSegment)
val recoveryPoint = offsetForRecoveryPointSegment + 1
assertTrue(recoveryPoint < offsetForSegmentAfterRecoveryPoint)
log.close()
val segmentsWithReads = mutable.Set[LogSegment]()
val recoveredSegments = mutable.Set[LogSegment]()
val expectedSegmentsWithReads = mutable.Set[Long]()
val expectedSnapshotOffsets = mutable.Set[Long]()
if (logConfig.messageFormatVersion.isLessThan(IBP_0_11_0_IV0)) {
expectedSegmentsWithReads += activeSegmentOffset
expectedSnapshotOffsets ++= log.logSegments.asScala.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset
} else {
expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Set(activeSegmentOffset)
expectedSnapshotOffsets ++= log.logSegments.asScala.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset
}
def createLogWithInterceptedReads(recoveryPoint: Long): UnifiedLog = {
val maxTransactionTimeoutMs = 5 * 60 * 1000
val producerIdExpirationCheckIntervalMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logDirFailureChannel = new LogDirFailureChannel(10)
// Intercept all segment read calls
val interceptedLogSegments = new LogSegments(topicPartition) {
override def add(segment: LogSegment): LogSegment = {
val wrapper = Mockito.spy(segment)
Mockito.doAnswer { in =>
segmentsWithReads += wrapper
segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.lang.Long]), in.getArgument(3, classOf[java.lang.Boolean]))
}.when(wrapper).read(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())
Mockito.doAnswer { in =>
recoveredSegments += wrapper
segment.recover(in.getArgument(0, classOf[ProducerStateManager]), in.getArgument(1, classOf[Optional[LeaderEpochFileCache]]))
}.when(wrapper).recover(ArgumentMatchers.any(), ArgumentMatchers.any())
super.add(wrapper)
}
}
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "")
val producerStateManager = new ProducerStateManager(topicPartition, logDir,
maxTransactionTimeoutMs, producerStateManagerConfig, mockTime)
val logLoader = new LogLoader(
logDir,
topicPartition,
logConfig,
mockTime.scheduler,
mockTime,
logDirFailureChannel,
hadCleanShutdown = false,
interceptedLogSegments,
0L,
recoveryPoint,
leaderEpochCache.asJava,
producerStateManager)
val offsets = logLoader.load()
val localLog = new LocalLog(logDir, logConfig, interceptedLogSegments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition,
logDirFailureChannel)
new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats,
producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager,
None, keepPartitionMetadataFile = true)
}
// Retain snapshots for the last 2 segments
log.producerStateManager.deleteSnapshotsBefore(segmentOffsets(segmentOffsets.size - 2))
log = createLogWithInterceptedReads(offsetForRecoveryPointSegment)
// We will reload all segments because the recovery point is behind the producer snapshot files (pre KAFKA-5829 behaviour)
assertEquals(expectedSegmentsWithReads, segmentsWithReads.map(_.baseOffset))
assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset))
assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir).toSet)
log.close()
segmentsWithReads.clear()
recoveredSegments.clear()
// Only delete snapshots before the base offset of the recovery point segment (post KAFKA-5829 behaviour) to
// avoid reading all segments
log.producerStateManager.deleteSnapshotsBefore(offsetForRecoveryPointSegment)
log = createLogWithInterceptedReads(recoveryPoint = recoveryPoint)
assertEquals(Set(activeSegmentOffset), segmentsWithReads.map(_.baseOffset))
assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset))
assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir).toSet)
log.close()
}
@Test
def testSkipLoadingIfEmptyProducerStateBeforeTruncation(): Unit = {
val maxTransactionTimeoutMs = 60000
val producerStateManagerConfig = new ProducerStateManagerConfig(300000, false)
val stateManager: ProducerStateManager = mock(classOf[ProducerStateManager])
when(stateManager.producerStateManagerConfig).thenReturn(producerStateManagerConfig)
when(stateManager.maxTransactionTimeoutMs).thenReturn(maxTransactionTimeoutMs)
when(stateManager.latestSnapshotOffset).thenReturn(OptionalLong.empty())
when(stateManager.mapEndOffset).thenReturn(0L)
when(stateManager.isEmpty).thenReturn(true)
when(stateManager.firstUnstableOffset).thenReturn(Optional.empty[LogOffsetMetadata]())
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1)
val config = new LogConfig(new Properties())
val segments = new LogSegments(topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val offsets = new LogLoader(
logDir,
topicPartition,
config,
mockTime.scheduler,
mockTime,
logDirFailureChannel,
hadCleanShutdown = false,
segments,
0L,
0L,
leaderEpochCache.asJava,
stateManager
).load()
val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition,
logDirFailureChannel)
val log = new UnifiedLog(offsets.logStartOffset,
localLog,
brokerTopicStats = brokerTopicStats,
producerIdExpirationCheckIntervalMs = 30000,
leaderEpochCache = leaderEpochCache,
producerStateManager = stateManager,
_topicId = None,
keepPartitionMetadataFile = true)
verify(stateManager).updateMapEndOffset(0L)
verify(stateManager).removeStraySnapshots(any())
verify(stateManager).takeSnapshot()
verify(stateManager).truncateAndReload(ArgumentMatchers.eq(0L), ArgumentMatchers.eq(0L), anyLong)
// Append some messages
reset(stateManager)
when(stateManager.firstUnstableOffset).thenReturn(Optional.empty[LogOffsetMetadata]())
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0)
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0)
verify(stateManager).updateMapEndOffset(1L)
verify(stateManager).updateMapEndOffset(2L)
// Now truncate
reset(stateManager)
when(stateManager.firstUnstableOffset).thenReturn(Optional.empty[LogOffsetMetadata]())
when(stateManager.latestSnapshotOffset).thenReturn(OptionalLong.empty())
when(stateManager.isEmpty).thenReturn(true)
when(stateManager.mapEndOffset).thenReturn(2L)
// Truncation causes the map end offset to reset to 0
when(stateManager.mapEndOffset).thenReturn(0L)
log.truncateTo(1L)
verify(stateManager).truncateAndReload(ArgumentMatchers.eq(0L), ArgumentMatchers.eq(1L), anyLong)
verify(stateManager).updateMapEndOffset(1L)
verify(stateManager, times(2)).takeSnapshot()
}
@Test
def testRecoverAfterNonMonotonicCoordinatorEpochWrite(): Unit = {
// Due to KAFKA-9144, we may encounter a coordinator epoch which goes backwards.
// This test case verifies that recovery logic relaxes validation in this case and
// just takes the latest write.
val producerId = 1L
val coordinatorEpoch = 5
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5)
var log = createLog(logDir, logConfig)
val epoch = 0.toShort
val firstAppendTimestamp = mockTime.milliseconds()
LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT,
firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch)
assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp)
val maxProducerIdExpirationMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_DEFAULT
mockTime.sleep(maxProducerIdExpirationMs)
assertEquals(Optional.empty(), log.producerStateManager.lastEntry(producerId))
val secondAppendTimestamp = mockTime.milliseconds()
LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT,
secondAppendTimestamp, coordinatorEpoch = coordinatorEpoch - 1)
log.close()
// Force recovery by setting the recoveryPoint to the log start
log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false)
assertEquals(secondAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp)
log.close()
}
@nowarn("cat=deprecation")
@Test
def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown(): Unit = {
val maxTransactionTimeoutMs = 60000
val producerStateManagerConfig = new ProducerStateManagerConfig(300000, false)
val stateManager: ProducerStateManager = mock(classOf[ProducerStateManager])
when(stateManager.isEmpty).thenReturn(true)
when(stateManager.firstUnstableOffset).thenReturn(Optional.empty[LogOffsetMetadata]())
when(stateManager.producerStateManagerConfig).thenReturn(producerStateManagerConfig)
when(stateManager.maxTransactionTimeoutMs).thenReturn(maxTransactionTimeoutMs)
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logProps = new Properties()
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2")
val config = new LogConfig(logProps)
val logDirFailureChannel = null
val segments = new LogSegments(topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val offsets = new LogLoader(
logDir,
topicPartition,
config,
mockTime.scheduler,
mockTime,
logDirFailureChannel,
hadCleanShutdown = false,
segments,
0L,
0L,
leaderEpochCache.asJava,
stateManager
).load()
val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition,
logDirFailureChannel)
new UnifiedLog(offsets.logStartOffset,
localLog,
brokerTopicStats = brokerTopicStats,
producerIdExpirationCheckIntervalMs = 30000,
leaderEpochCache = leaderEpochCache,
producerStateManager = stateManager,
_topicId = None,
keepPartitionMetadataFile = true)
verify(stateManager).removeStraySnapshots(any[java.util.List[java.lang.Long]])
verify(stateManager, times(2)).updateMapEndOffset(0L)
verify(stateManager, times(2)).takeSnapshot()
verify(stateManager).isEmpty
verify(stateManager).firstUnstableOffset
verify(stateManager, times(2)).takeSnapshot()
verify(stateManager, times(2)).updateMapEndOffset(0L)
}
@nowarn("cat=deprecation")
@Test
def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown(): Unit = {
val maxTransactionTimeoutMs = 60000
val producerStateManagerConfig = new ProducerStateManagerConfig(300000, false)
val stateManager: ProducerStateManager = mock(classOf[ProducerStateManager])
when(stateManager.isEmpty).thenReturn(true)
when(stateManager.firstUnstableOffset).thenReturn(Optional.empty[LogOffsetMetadata]())
when(stateManager.producerStateManagerConfig).thenReturn(producerStateManagerConfig)
when(stateManager.maxTransactionTimeoutMs).thenReturn(maxTransactionTimeoutMs)
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logProps = new Properties()
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2")
val config = new LogConfig(logProps)
val logDirFailureChannel = null
val segments = new LogSegments(topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val offsets = new LogLoader(
logDir,
topicPartition,
config,
mockTime.scheduler,
mockTime,
logDirFailureChannel,
hadCleanShutdown = true,
segments,
0L,
0L,
leaderEpochCache.asJava,
stateManager
).load()
val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition,
logDirFailureChannel)
new UnifiedLog(offsets.logStartOffset,
localLog,
brokerTopicStats = brokerTopicStats,
producerIdExpirationCheckIntervalMs = 30000,
leaderEpochCache = leaderEpochCache,
producerStateManager = stateManager,
_topicId = None,
keepPartitionMetadataFile = true)
verify(stateManager).removeStraySnapshots(any[java.util.List[java.lang.Long]])
verify(stateManager, times(2)).updateMapEndOffset(0L)
verify(stateManager, times(2)).takeSnapshot()
verify(stateManager).isEmpty
verify(stateManager).firstUnstableOffset
}
@nowarn("cat=deprecation")
@Test
def testSkipTruncateAndReloadIfNewMessageFormatAndCleanShutdown(): Unit = {
val maxTransactionTimeoutMs = 60000
val producerStateManagerConfig = new ProducerStateManagerConfig(300000, false)
val stateManager: ProducerStateManager = mock(classOf[ProducerStateManager])
when(stateManager.latestSnapshotOffset).thenReturn(OptionalLong.empty())
when(stateManager.isEmpty).thenReturn(true)
when(stateManager.firstUnstableOffset).thenReturn(Optional.empty[LogOffsetMetadata]())
when(stateManager.producerStateManagerConfig).thenReturn(producerStateManagerConfig)
when(stateManager.maxTransactionTimeoutMs).thenReturn(maxTransactionTimeoutMs)
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logProps = new Properties()
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
val config = new LogConfig(logProps)
val logDirFailureChannel = null
val segments = new LogSegments(topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val offsets = new LogLoader(
logDir,
topicPartition,
config,
mockTime.scheduler,
mockTime,
logDirFailureChannel,
hadCleanShutdown = true,
segments,
0L,
0L,
leaderEpochCache.asJava,
stateManager
).load()
val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition,
logDirFailureChannel)
new UnifiedLog(offsets.logStartOffset,
localLog,
brokerTopicStats = brokerTopicStats,
producerIdExpirationCheckIntervalMs = 30000,
leaderEpochCache = leaderEpochCache,
producerStateManager = stateManager,
_topicId = None,
keepPartitionMetadataFile = true)
verify(stateManager).removeStraySnapshots(any[java.util.List[java.lang.Long]])
verify(stateManager, times(2)).updateMapEndOffset(0L)
verify(stateManager, times(2)).takeSnapshot()
verify(stateManager).isEmpty
verify(stateManager).firstUnstableOffset
}
@Test
def testLoadProducersAfterDeleteRecordsMidSegment(): Unit = {
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5)
val log = createLog(logDir, logConfig)
val pid1 = 1L
val pid2 = 2L
val epoch = 0.toShort
log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1,
producerEpoch = epoch, sequence = 0), leaderEpoch = 0)
log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2,
producerEpoch = epoch, sequence = 0), leaderEpoch = 0)
assertEquals(2, log.activeProducersWithLastSequence.size)
log.updateHighWatermark(log.logEndOffset)
log.maybeIncrementLogStartOffset(1L, LogStartOffsetIncrementReason.ClientRecordDeletion)
// Deleting records should not remove producer state
assertEquals(2, log.activeProducersWithLastSequence.size)
val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2)
assertTrue(retainedLastSeqOpt.isDefined)
assertEquals(0, retainedLastSeqOpt.get)
log.close()
// Because the log start offset did not advance, producer snapshots will still be present and the state will be rebuilt
val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false)
assertEquals(2, reloadedLog.activeProducersWithLastSequence.size)
val reloadedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2)
assertEquals(retainedLastSeqOpt, reloadedLastSeqOpt)
}
@Test
def testLoadingLogKeepsLargestStrayProducerStateSnapshot(): Unit = {
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0)
val log = createLog(logDir, logConfig)
val pid1 = 1L
val epoch = 0.toShort
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 0), leaderEpoch = 0)
log.roll()
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 1), leaderEpoch = 0)
log.roll()
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("c".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 2), leaderEpoch = 0)
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("d".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 3), leaderEpoch = 0)
// Close the log, we should now have 3 segments
log.close()
assertEquals(log.logSegments.size, 3)
// We expect 3 snapshot files, two of which are for the first two segments, the last was written out during log closing.
assertEquals(Seq(1L, 2L, 4L), ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted)
// Inject a stray snapshot file within the bounds of the log at offset 3, it should be cleaned up after loading the log
val straySnapshotFile = LogFileUtils.producerSnapshotFile(logDir, 3).toPath
Files.createFile(straySnapshotFile)
assertEquals(Seq(1L, 2L, 3L, 4L), ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted)
createLog(logDir, logConfig, lastShutdownClean = false)
// We should clean up the stray producer state snapshot file, but keep the largest snapshot file (4)
assertEquals(Seq(1L, 2L, 4L), ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted)
}
@Test
def testLoadProducersAfterDeleteRecordsOnSegment(): Unit = {
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5)
val log = createLog(logDir, logConfig)
val pid1 = 1L
val pid2 = 2L
val epoch = 0.toShort
log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1,
producerEpoch = epoch, sequence = 0), leaderEpoch = 0)
log.roll()
log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2,
producerEpoch = epoch, sequence = 0), leaderEpoch = 0)
assertEquals(2, log.logSegments.size)
assertEquals(2, log.activeProducersWithLastSequence.size)
log.updateHighWatermark(log.logEndOffset)
log.maybeIncrementLogStartOffset(1L, LogStartOffsetIncrementReason.ClientRecordDeletion)
log.deleteOldSegments()
// Deleting records should not remove producer state
assertEquals(1, log.logSegments.size)
assertEquals(2, log.activeProducersWithLastSequence.size)
val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2)
assertTrue(retainedLastSeqOpt.isDefined)
assertEquals(0, retainedLastSeqOpt.get)
log.close()
// After reloading log, producer state should not be regenerated
val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false)
assertEquals(1, reloadedLog.activeProducersWithLastSequence.size)
val reloadedEntryOpt = log.activeProducersWithLastSequence.get(pid2)
assertEquals(retainedLastSeqOpt, reloadedEntryOpt)
}
/**
* Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly.
*/
@Test
def testLogRecoversToCorrectOffset(): Unit = {
val numMessages = 100
val messageSize = 100
val segmentSize = 7 * messageSize
val indexInterval = 3 * messageSize
val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096)
var log = createLog(logDir, logConfig)
for (i <- 0 until numMessages)
log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize),
timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0)
assertEquals(numMessages, log.logEndOffset,
"After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages))
val lastIndexOffset = log.activeSegment.offsetIndex.lastOffset
val numIndexEntries = log.activeSegment.offsetIndex.entries
val lastOffset = log.logEndOffset
// After segment is closed, the last entry in the time index should be (largest timestamp -> last offset).
val lastTimeIndexOffset = log.logEndOffset - 1
val lastTimeIndexTimestamp = log.activeSegment.largestTimestamp
// Depending on when the last time index entry is inserted, an entry may or may not be inserted into the time index.
val numTimeIndexEntries = log.activeSegment.timeIndex.entries + {
if (log.activeSegment.timeIndex.lastEntry.offset == log.logEndOffset - 1) 0 else 1
}
log.close()
def verifyRecoveredLog(log: UnifiedLog, expectedRecoveryPoint: Long): Unit = {
assertEquals(expectedRecoveryPoint, log.recoveryPoint, s"Unexpected recovery point")
assertEquals(numMessages, log.logEndOffset, s"Should have $numMessages messages when log is reopened w/o recovery")
assertEquals(lastIndexOffset, log.activeSegment.offsetIndex.lastOffset, "Should have same last index offset as before.")
assertEquals(numIndexEntries, log.activeSegment.offsetIndex.entries, "Should have same number of index entries as before.")
assertEquals(lastTimeIndexTimestamp, log.activeSegment.timeIndex.lastEntry.timestamp, "Should have same last time index timestamp")
assertEquals(lastTimeIndexOffset, log.activeSegment.timeIndex.lastEntry.offset, "Should have same last time index offset")
assertEquals(numTimeIndexEntries, log.activeSegment.timeIndex.entries, "Should have same number of time index entries as before.")
}
log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false)
verifyRecoveredLog(log, lastOffset)
log.close()
// test recovery case
val recoveryPoint = 10
log = createLog(logDir, logConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false)
// the recovery point should not be updated after unclean shutdown until the log is flushed
verifyRecoveredLog(log, recoveryPoint)
log.flush(false)
verifyRecoveredLog(log, lastOffset)
log.close()
}
/**
* Test that if we manually delete an index segment it is rebuilt when the log is re-opened
*/
@Test
def testIndexRebuild(): Unit = {
// publish the messages and close the log
val numMessages = 200
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1)
var log = createLog(logDir, logConfig)
for (i <- 0 until numMessages)
log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0)
val indexFiles = log.logSegments.asScala.map(_.offsetIndexFile)
val timeIndexFiles = log.logSegments.asScala.map(_.timeIndexFile)
log.close()
// delete all the index files
indexFiles.foreach(_.delete())
timeIndexFiles.foreach(_.delete())
// reopen the log
log = createLog(logDir, logConfig, lastShutdownClean = false)
assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages))
assertTrue(log.logSegments.asScala.head.offsetIndex.entries > 0, "The index should have been rebuilt")
assertTrue(log.logSegments.asScala.head.timeIndex.entries > 0, "The time index should have been rebuilt")
for (i <- 0 until numMessages) {
assertEquals(i, LogTestUtils.readLog(log, i, 100).records.batches.iterator.next().lastOffset)
if (i == 0)
assertEquals(log.logSegments.asScala.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset)
else
assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset)
}
log.close()
}
/**
* Test that if messages format version of the messages in a segment is before 0.10.0, the time index should be empty.
*/
@nowarn("cat=deprecation")
@Test
def testRebuildTimeIndexForOldMessages(): Unit = {
val numMessages = 200
val segmentSize = 200
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize.toString)
logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1")
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0")
val logConfig = new LogConfig(logProps)
var log = createLog(logDir, logConfig)
for (i <- 0 until numMessages)
log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10),
timestamp = mockTime.milliseconds + i * 10, magicValue = RecordBatch.MAGIC_VALUE_V1), leaderEpoch = 0)
val timeIndexFiles = log.logSegments.asScala.map(_.timeIndexFile())
log.close()
// Delete the time index.
timeIndexFiles.foreach(file => Files.delete(file.toPath))
// The rebuilt time index should be empty
log = createLog(logDir, logConfig, recoveryPoint = numMessages + 1, lastShutdownClean = false)
for (segment <- log.logSegments.asScala.init) {
assertEquals(0, segment.timeIndex.entries, "The time index should be empty")
assertEquals(0, segment.timeIndexFile().length, "The time index file size should be 0")
}
}
/**
* Test that if we have corrupted an index segment it is rebuilt when the log is re-opened
*/
@Test
def testCorruptIndexRebuild(): Unit = {
// publish the messages and close the log
val numMessages = 200
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1)
var log = createLog(logDir, logConfig)
for (i <- 0 until numMessages)
log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0)
val indexFiles = log.logSegments.asScala.map(_.offsetIndexFile())
val timeIndexFiles = log.logSegments.asScala.map(_.timeIndexFile())
log.close()
// corrupt all the index files
for ( file <- indexFiles) {
val bw = new BufferedWriter(new FileWriter(file))
bw.write(" ")
bw.close()
}
// corrupt all the index files
for ( file <- timeIndexFiles) {
val bw = new BufferedWriter(new FileWriter(file))
bw.write(" ")
bw.close()
}
// reopen the log with recovery point=0 so that the segment recovery can be triggered
log = createLog(logDir, logConfig, lastShutdownClean = false)
assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages))
for (i <- 0 until numMessages) {
assertEquals(i, LogTestUtils.readLog(log, i, 100).records.batches.iterator.next().lastOffset)
if (i == 0)
assertEquals(log.logSegments.asScala.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset)
else
assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset)
}
log.close()
}
/**
* When we open a log any index segments without an associated log segment should be deleted.
*/
@Test
def testBogusIndexSegmentsAreRemoved(): Unit = {
val bogusIndex1 = LogFileUtils.offsetIndexFile(logDir, 0)
val bogusTimeIndex1 = LogFileUtils.timeIndexFile(logDir, 0)
val bogusIndex2 = LogFileUtils.offsetIndexFile(logDir, 5)
val bogusTimeIndex2 = LogFileUtils.timeIndexFile(logDir, 5)
// The files remain absent until we first access it because we are doing lazy loading for time index and offset index
// files but in this test case we need to create these files in order to test we will remove them.
Files.createFile(bogusIndex2.toPath)
Files.createFile(bogusTimeIndex2.toPath)
def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds)
val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1)
val log = createLog(logDir, logConfig)
// Force the segment to access the index files because we are doing index lazy loading.
log.logSegments.asScala.head.offsetIndex
log.logSegments.asScala.head.timeIndex
assertTrue(bogusIndex1.length > 0,
"The first index file should have been replaced with a larger file")
assertTrue(bogusTimeIndex1.length > 0,
"The first time index file should have been replaced with a larger file")
assertFalse(bogusIndex2.exists,
"The second index file should have been deleted.")
assertFalse(bogusTimeIndex2.exists,
"The second time index file should have been deleted.")
// check that we can append to the log
for (_ <- 0 until 10)
log.appendAsLeader(createRecords, leaderEpoch = 0)
log.delete()
}
/**
* Verify that truncation works correctly after re-opening the log
*/
@Test
def testReopenThenTruncate(): Unit = {
def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds)
// create a log
val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000)
var log = createLog(logDir, logConfig)
// add enough messages to roll over several segments then close and re-open and attempt to truncate
for (_ <- 0 until 100)
log.appendAsLeader(createRecords, leaderEpoch = 0)
log.close()
log = createLog(logDir, logConfig, lastShutdownClean = false)
log.truncateTo(3)
assertEquals(1, log.numberOfSegments, "All but one segment should be deleted.")
assertEquals(3, log.logEndOffset, "Log end offset should be 3.")
}