forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
LogManagerTest.scala
executable file
·1435 lines (1226 loc) · 60 KB
/
LogManagerTest.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 com.yammer.metrics.core.{Gauge, MetricName}
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.server.metadata.{ConfigRepository, MockConfigRepository}
import kafka.server.BrokerTopicStats
import kafka.utils._
import org.apache.directory.api.util.FileUtils
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.OffsetOutOfRangeException
import org.apache.kafka.common.message.LeaderAndIsrRequestData
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrTopicState
import org.apache.kafka.common.requests.{AbstractControlRequest, LeaderAndIsrRequest}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{DirectoryId, KafkaException, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.image.{TopicImage, TopicsImage}
import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration}
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.mockito.ArgumentMatchers.any
import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
import org.mockito.Mockito.{doAnswer, doNothing, mock, never, spy, times, verify}
import java.io._
import java.nio.file.Files
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future}
import java.util.{Collections, Optional, OptionalLong, Properties}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, LogConfig, LogDirFailureChannel, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache}
import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler
import scala.collection.{Map, mutable}
import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
import scala.util.{Failure, Try}
class LogManagerTest {
import LogManagerTest._
val time = new MockTime()
val maxRollInterval = 100
val maxLogAgeMs: Int = 10 * 60 * 1000
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer)
logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer)
val logConfig = new LogConfig(logProps)
var logDir: File = _
var logManager: LogManager = _
val name = "kafka"
val veryLargeLogFlushInterval = 10000000L
val initialTaskDelayMs: Long = 10 * 1000
@BeforeEach
def setUp(): Unit = {
logDir = TestUtils.tempDir()
logManager = createLogManager()
logManager.startup(Set.empty)
assertEquals(initialTaskDelayMs, logManager.initialTaskDelayMs)
}
@AfterEach
def tearDown(): Unit = {
if (logManager != null)
logManager.shutdown()
Utils.delete(logDir)
// Some tests assign a new LogManager
if (logManager != null)
logManager.liveLogDirs.foreach(Utils.delete)
}
/**
* Test that getOrCreateLog on a non-existent log creates a new log and that we can append to the new log.
*/
@Test
def testCreateLog(): Unit = {
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
assertEquals(1, logManager.liveLogDirs.size)
val logFile = new File(logDir, name + "-0")
assertTrue(logFile.exists)
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
/**
* Test that getOrCreateLog on a non-existent log creates a new log in given logDirectory using directory id and that we can append to the new log.
*/
@Test
def testCreateLogOnTargetedLogDirectory(): Unit = {
val targetedLogDirectoryId = DirectoryId.random()
val dirs: Seq[File] = Seq.fill(5)(TestUtils.tempDir())
writeMetaProperties(dirs.head)
writeMetaProperties(dirs(1), Optional.of(targetedLogDirectoryId))
writeMetaProperties(dirs(3), Optional.of(DirectoryId.random()))
writeMetaProperties(dirs(4))
logManager = createLogManager(dirs)
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None, targetLogDirectoryId = Some(targetedLogDirectoryId))
assertEquals(5, logManager.liveLogDirs.size)
val logFile = new File(dirs(1), name + "-0")
assertTrue(logFile.exists)
assertEquals(dirs(1).getAbsolutePath, logFile.getParent)
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
/**
* Test that getOrCreateLog on a non-existent log creates a new log in the next selected logDirectory if the given directory id is DirectoryId.UNASSIGNED.
*/
@Test
def testCreateLogWithTargetedLogDirectorySetAsUnassigned(): Unit = {
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None, targetLogDirectoryId = Some(DirectoryId.UNASSIGNED))
assertEquals(1, logManager.liveLogDirs.size)
val logFile = new File(logDir, name + "-0")
assertTrue(logFile.exists)
assertFalse(logManager.directoryId(logFile.getParent).equals(DirectoryId.UNASSIGNED))
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
@Test
def testCreateLogWithTargetedLogDirectorySetAsUnknownWithoutAnyOfflineDirectories(): Unit = {
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None, targetLogDirectoryId = Some(DirectoryId.LOST))
assertEquals(1, logManager.liveLogDirs.size)
val logFile = new File(logDir, name + "-0")
assertTrue(logFile.exists)
assertFalse(logManager.directoryId(logFile.getParent).equals(DirectoryId.random()))
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
/**
* Tests that all internal futures are completed before LogManager.shutdown() returns to the
* caller during error situations.
*/
@Test
def testHandlingExceptionsDuringShutdown(): Unit = {
// We create two directories logDir1 and logDir2 to help effectively test error handling
// during LogManager.shutdown().
val logDir1 = TestUtils.tempDir()
val logDir2 = TestUtils.tempDir()
var logManagerForTest: Option[LogManager] = Option.empty
try {
logManagerForTest = Some(createLogManager(Seq(logDir1, logDir2)))
assertEquals(2, logManagerForTest.get.liveLogDirs.size)
logManagerForTest.get.startup(Set.empty)
val log1 = logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
val log2 = logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 1), topicId = None)
val logFile1 = new File(logDir1, name + "-0")
assertTrue(logFile1.exists)
val logFile2 = new File(logDir2, name + "-1")
assertTrue(logFile2.exists)
log1.appendAsLeader(TestUtils.singletonRecords("test1".getBytes()), leaderEpoch = 0)
log1.takeProducerSnapshot()
log1.appendAsLeader(TestUtils.singletonRecords("test1".getBytes()), leaderEpoch = 0)
log2.appendAsLeader(TestUtils.singletonRecords("test2".getBytes()), leaderEpoch = 0)
log2.takeProducerSnapshot()
log2.appendAsLeader(TestUtils.singletonRecords("test2".getBytes()), leaderEpoch = 0)
// This should cause log1.close() to fail during LogManger shutdown sequence.
FileUtils.deleteDirectory(logFile1)
logManagerForTest.get.shutdown(3)
assertFalse(Files.exists(new File(logDir1, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).toPath))
assertTrue(Files.exists(new File(logDir2, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).toPath))
assertEquals(OptionalLong.empty(), logManagerForTest.get.readBrokerEpochFromCleanShutdownFiles())
} finally {
logManagerForTest.foreach(manager => manager.liveLogDirs.foreach(Utils.delete))
}
}
@Test
def testCleanShutdownFileWithBrokerEpoch(): Unit = {
val logDir1 = TestUtils.tempDir()
val logDir2 = TestUtils.tempDir()
var logManagerForTest: Option[LogManager] = Option.empty
try {
logManagerForTest = Some(createLogManager(Seq(logDir1, logDir2)))
assertEquals(2, logManagerForTest.get.liveLogDirs.size)
logManagerForTest.get.startup(Set.empty)
logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 1), topicId = None)
val logFile1 = new File(logDir1, name + "-0")
assertTrue(logFile1.exists)
val logFile2 = new File(logDir2, name + "-1")
assertTrue(logFile2.exists)
logManagerForTest.get.shutdown(3)
assertTrue(Files.exists(new File(logDir1, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).toPath))
assertTrue(Files.exists(new File(logDir2, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).toPath))
assertEquals(OptionalLong.of(3L), logManagerForTest.get.readBrokerEpochFromCleanShutdownFiles())
} finally {
logManagerForTest.foreach(manager => manager.liveLogDirs.foreach(Utils.delete))
}
}
/*
* Test that LogManager.shutdown() doesn't create clean shutdown file for a log directory that has not completed
* recovery.
*/
@Test
def testCleanShutdownFileWhenShutdownCalledBeforeStartupComplete(): Unit = {
// 1. create two logs under logDir
val topicPartition0 = new TopicPartition(name, 0)
val topicPartition1 = new TopicPartition(name, 1)
val log0 = logManager.getOrCreateLog(topicPartition0, topicId = None)
val log1 = logManager.getOrCreateLog(topicPartition1, topicId = None)
val logFile0 = new File(logDir, name + "-0")
val logFile1 = new File(logDir, name + "-1")
assertTrue(logFile0.exists)
assertTrue(logFile1.exists)
log0.appendAsLeader(TestUtils.singletonRecords("test1".getBytes()), leaderEpoch = 0)
log0.takeProducerSnapshot()
log1.appendAsLeader(TestUtils.singletonRecords("test1".getBytes()), leaderEpoch = 0)
log1.takeProducerSnapshot()
// 2. simulate unclean shutdown by deleting clean shutdown marker file
logManager.shutdown()
assertTrue(Files.deleteIfExists(new File(logDir, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).toPath))
// 3. create a new LogManager and start it in a different thread
@volatile var loadLogCalled = 0
logManager = spy(createLogManager())
doAnswer { invocation =>
// intercept LogManager.loadLog to sleep 5 seconds so that there is enough time to call LogManager.shutdown
// before LogManager.startup completes.
Thread.sleep(5000)
invocation.callRealMethod().asInstanceOf[UnifiedLog]
loadLogCalled = loadLogCalled + 1
}.when(logManager).loadLog(any[File], any[Boolean], any[Map[TopicPartition, Long]], any[Map[TopicPartition, Long]],
any[LogConfig], any[Map[String, LogConfig]], any[ConcurrentMap[String, Int]], any[UnifiedLog => Boolean]())
val t = new Thread() {
override def run(): Unit = { logManager.startup(Set.empty) }
}
t.start()
// 4. shutdown LogManager after the first log is loaded but before the second log is loaded
TestUtils.waitUntilTrue(() => loadLogCalled == 1,
"Timed out waiting for only the first log to be loaded")
logManager.shutdown()
logManager = null
// 5. verify that CleanShutdownFile is not created under logDir
assertFalse(Files.exists(new File(logDir, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).toPath))
}
/**
* Test that getOrCreateLog on a non-existent log creates a new log and that we can append to the new log.
* The LogManager is configured with one invalid log directory which should be marked as offline.
*/
@Test
def testCreateLogWithInvalidLogDir(): Unit = {
// Configure the log dir with the Nul character as the path, which causes dir.getCanonicalPath() to throw an
// IOException. This simulates the scenario where the disk is not properly mounted (which is hard to achieve in
// a unit test)
val dirs = Seq(logDir, new File("\u0000"))
logManager.shutdown()
logManager = createLogManager(dirs)
logManager.startup(Set.empty)
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), isNew = true, topicId = None)
val logFile = new File(logDir, name + "-0")
assertTrue(logFile.exists)
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
@Test
def testCreateLogWithLogDirFallback(): Unit = {
// Configure a number of directories one level deeper in logDir,
// so they all get cleaned up in tearDown().
val dirs = (0 to 4)
.map(_.toString)
.map(logDir.toPath.resolve(_).toFile)
// Create a new LogManager with the configured directories and an overridden createLogDirectory.
logManager.shutdown()
logManager = spy(createLogManager(dirs))
val brokenDirs = mutable.Set[File]()
doAnswer { invocation =>
// The first half of directories tried will fail, the rest goes through.
val logDir = invocation.getArgument[File](0)
if (brokenDirs.contains(logDir) || brokenDirs.size < dirs.length / 2) {
brokenDirs.add(logDir)
Failure(new Throwable("broken dir"))
} else {
invocation.callRealMethod().asInstanceOf[Try[File]]
}
}.when(logManager).createLogDirectory(any(), any())
logManager.startup(Set.empty)
// Request creating a new log.
// LogManager should try using all configured log directories until one succeeds.
logManager.getOrCreateLog(new TopicPartition(name, 0), isNew = true, topicId = None)
// Verify that half the directories were considered broken,
assertEquals(dirs.length / 2, brokenDirs.size)
// and that exactly one log file was created,
val containsLogFile: File => Boolean = dir => new File(dir, name + "-0").exists()
assertEquals(1, dirs.count(containsLogFile), "More than one log file created")
// and that it wasn't created in one of the broken directories.
assertFalse(brokenDirs.exists(containsLogFile))
}
/**
* Test that get on a non-existent returns None and no log is created.
*/
@Test
def testGetNonExistentLog(): Unit = {
val log = logManager.getLog(new TopicPartition(name, 0))
assertEquals(None, log, "No log should be found.")
val logFile = new File(logDir, name + "-0")
assertFalse(logFile.exists)
}
/**
* Test time-based log cleanup. First append messages, then set the time into the future and run cleanup.
*/
@Test
def testCleanupExpiredSegments(): Unit = {
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
var offset = 0L
for (_ <- 0 until 200) {
val set = TestUtils.singletonRecords("test".getBytes())
val info = log.appendAsLeader(set, leaderEpoch = 0)
offset = info.lastOffset
}
assertTrue(log.numberOfSegments > 1, "There should be more than one segment now.")
log.updateHighWatermark(log.logEndOffset)
log.logSegments.forEach(_.log.file.setLastModified(time.milliseconds))
time.sleep(maxLogAgeMs + 1)
assertEquals(1, log.numberOfSegments, "Now there should only be only one segment in the index.")
time.sleep(log.config.fileDeleteDelayMs + 1)
log.logSegments.forEach(s => {
s.offsetIndex()
s.timeIndex()
})
// there should be a log file, two indexes, one producer snapshot, and the leader epoch checkpoint
assertEquals(log.numberOfSegments * 4 + 1, log.dir.list.length, "Files should have been deleted")
assertEquals(0, readLog(log, offset + 1).records.sizeInBytes, "Should get empty fetch off new log.")
assertThrows(classOf[OffsetOutOfRangeException], () => readLog(log, 0), () => "Should get exception from fetching earlier.")
// log should still be appendable
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
/**
* Test size-based cleanup. Append messages, then run cleanup and check that segments are deleted.
*/
@Test
def testCleanupSegmentsToMaintainSize(): Unit = {
val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes
logManager.shutdown()
val segmentBytes = 10 * setSize
val properties = new Properties()
properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString)
properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString)
val configRepository = MockConfigRepository.forTopic(name, properties)
logManager = createLogManager(configRepository = configRepository)
logManager.startup(Set.empty)
// create a log
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
var offset = 0L
// add a bunch of messages that should be larger than the retentionSize
val numMessages = 200
for (_ <- 0 until numMessages) {
val set = TestUtils.singletonRecords("test".getBytes())
val info = log.appendAsLeader(set, leaderEpoch = 0)
offset = info.firstOffset
}
log.updateHighWatermark(log.logEndOffset)
assertEquals(numMessages * setSize / segmentBytes, log.numberOfSegments, "Check we have the expected number of segments.")
// this cleanup shouldn't find any expired segments but should delete some to reduce size
time.sleep(logManager.initialTaskDelayMs)
assertEquals(6, log.numberOfSegments, "Now there should be exactly 6 segments")
time.sleep(log.config.fileDeleteDelayMs + 1)
// there should be a log file, two indexes (the txn index is created lazily),
// and a producer snapshot file per segment, and the leader epoch checkpoint.
assertEquals(log.numberOfSegments * 4 + 1, log.dir.list.length, "Files should have been deleted")
assertEquals(0, readLog(log, offset + 1).records.sizeInBytes, "Should get empty fetch off new log.")
assertThrows(classOf[OffsetOutOfRangeException], () => readLog(log, 0))
// log should still be appendable
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
}
/**
* Ensures that LogManager doesn't run on logs with cleanup.policy=compact,delete
* LogCleaner.CleanerThread handles all logs where compaction is enabled.
*/
@Test
def testDoesntCleanLogsWithCompactDeletePolicy(): Unit = {
testDoesntCleanLogs(TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE)
}
/**
* Ensures that LogManager doesn't run on logs with cleanup.policy=compact
* LogCleaner.CleanerThread handles all logs where compaction is enabled.
*/
@Test
def testDoesntCleanLogsWithCompactPolicy(): Unit = {
testDoesntCleanLogs(TopicConfig.CLEANUP_POLICY_COMPACT)
}
private def testDoesntCleanLogs(policy: String): Unit = {
logManager.shutdown()
val configRepository = MockConfigRepository.forTopic(name, TopicConfig.CLEANUP_POLICY_CONFIG, policy)
logManager = createLogManager(configRepository = configRepository)
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
var offset = 0L
for (_ <- 0 until 200) {
val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes())
val info = log.appendAsLeader(set, leaderEpoch = 0)
offset = info.lastOffset
}
val numSegments = log.numberOfSegments
assertTrue(log.numberOfSegments > 1, "There should be more than one segment now.")
log.logSegments.forEach(_.log.file.setLastModified(time.milliseconds))
time.sleep(maxLogAgeMs + 1)
assertEquals(numSegments, log.numberOfSegments, "number of segments shouldn't have changed")
}
/**
* Test that flush is invoked by the background scheduler thread.
*/
@Test
def testTimeBasedFlush(): Unit = {
logManager.shutdown()
val configRepository = MockConfigRepository.forTopic(name, TopicConfig.FLUSH_MS_CONFIG, "1000")
logManager = createLogManager(configRepository = configRepository)
logManager.startup(Set.empty)
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
val lastFlush = log.lastFlushTime
for (_ <- 0 until 200) {
val set = TestUtils.singletonRecords("test".getBytes())
log.appendAsLeader(set, leaderEpoch = 0)
}
time.sleep(logManager.initialTaskDelayMs)
assertTrue(lastFlush != log.lastFlushTime, "Time based flush should have been triggered")
}
/**
* Test that new logs that are created are assigned to the least loaded log directory
*/
@Test
def testLeastLoadedAssignment(): Unit = {
// create a log manager with multiple data directories
val dirs = Seq(TestUtils.tempDir(),
TestUtils.tempDir(),
TestUtils.tempDir())
logManager.shutdown()
logManager = createLogManager(dirs)
// verify that logs are always assigned to the least loaded partition
for (partition <- 0 until 20) {
logManager.getOrCreateLog(new TopicPartition("test", partition), topicId = None)
assertEquals(partition + 1, logManager.allLogs.size, "We should have created the right number of logs")
val counts = logManager.allLogs.groupBy(_.dir.getParent).values.map(_.size)
assertTrue(counts.max <= counts.min + 1, "Load should balance evenly")
}
}
/**
* Tests that the log manager skips the remote-log-index-cache directory when loading the logs from disk
*/
@Test
def testLoadLogsSkipRemoteIndexCache(): Unit = {
val logDir = TestUtils.tempDir()
val remoteIndexCache = new File(logDir, RemoteIndexCache.DIR_NAME)
remoteIndexCache.mkdir()
logManager = createLogManager(Seq(logDir))
logManager.loadLogs(logConfig, Map.empty, _ => false)
}
@Test
def testLoadLogRenameLogThatShouldBeStray(): Unit = {
var invokedCount = 0
val logDir = TestUtils.tempDir()
logManager = createLogManager(Seq(logDir))
val testTopic = "test-stray-topic"
val testTopicPartition = new TopicPartition(testTopic, 0)
val log = logManager.getOrCreateLog(testTopicPartition, topicId = Some(Uuid.randomUuid()))
def providedIsStray(log: UnifiedLog) = {
invokedCount += 1
true
}
logManager.loadLog(log.dir, true, Map.empty, Map.empty, logConfig, Map.empty, new ConcurrentHashMap[String, Int](), providedIsStray)
assertEquals(1, invokedCount)
assertTrue(
logDir.listFiles().toSet
.exists(f => f.getName.startsWith(testTopic) && f.getName.endsWith(UnifiedLog.StrayDirSuffix))
)
}
/**
* Test that it is not possible to open two log managers using the same data directory
*/
@Test
def testTwoLogManagersUsingSameDirFails(): Unit = {
assertThrows(classOf[KafkaException], () => createLogManager())
}
/**
* Test that recovery points are correctly written out to disk
*/
@Test
def testCheckpointRecoveryPoints(): Unit = {
verifyCheckpointRecovery(Seq(new TopicPartition("test-a", 1), new TopicPartition("test-b", 1)), logManager, logDir)
}
/**
* Test that recovery points directory checking works with trailing slash
*/
@Test
def testRecoveryDirectoryMappingWithTrailingSlash(): Unit = {
logManager.shutdown()
logManager = TestUtils.createLogManager(logDirs = Seq(new File(TestUtils.tempDir().getAbsolutePath + File.separator)))
logManager.startup(Set.empty)
verifyCheckpointRecovery(Seq(new TopicPartition("test-a", 1)), logManager, logManager.liveLogDirs.head)
}
/**
* Test that recovery points directory checking works with relative directory
*/
@Test
def testRecoveryDirectoryMappingWithRelativeDirectory(): Unit = {
logManager.shutdown()
logManager = createLogManager(Seq(new File("data", logDir.getName).getAbsoluteFile))
logManager.startup(Set.empty)
verifyCheckpointRecovery(Seq(new TopicPartition("test-a", 1)), logManager, logManager.liveLogDirs.head)
}
private def verifyCheckpointRecovery(topicPartitions: Seq[TopicPartition], logManager: LogManager, logDir: File): Unit = {
val logs = topicPartitions.map(logManager.getOrCreateLog(_, topicId = None))
logs.foreach { log =>
for (_ <- 0 until 50)
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
log.flush(false)
}
logManager.checkpointLogRecoveryOffsets()
val checkpoints = new OffsetCheckpointFile(new File(logDir, LogManager.RecoveryPointCheckpointFile)).read()
topicPartitions.zip(logs).foreach { case (tp, log) =>
assertEquals(checkpoints(tp), log.recoveryPoint, "Recovery point should equal checkpoint")
}
}
private def createLogManager(logDirs: Seq[File] = Seq(this.logDir),
configRepository: ConfigRepository = new MockConfigRepository,
recoveryThreadsPerDataDir: Int = 1): LogManager = {
TestUtils.createLogManager(
defaultConfig = logConfig,
configRepository = configRepository,
logDirs = logDirs,
time = this.time,
recoveryThreadsPerDataDir = recoveryThreadsPerDataDir,
initialTaskDelayMs = initialTaskDelayMs)
}
@Test
def testFileReferencesAfterAsyncDelete(): Unit = {
val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None)
val activeSegment = log.activeSegment
val logName = activeSegment.log.file.getName
val indexName = activeSegment.offsetIndex.file.getName
val timeIndexName = activeSegment.timeIndex.file.getName
val txnIndexName = activeSegment.txnIndex.file.getName
val indexFilesOnDiskBeforeDelete = activeSegment.log.file.getParentFile.listFiles.filter(_.getName.endsWith("index"))
val removedLog = logManager.asyncDelete(new TopicPartition(name, 0)).get
val removedSegment = removedLog.activeSegment
val indexFilesAfterDelete = Seq(removedSegment.offsetIndexFile, removedSegment.timeIndexFile,
removedSegment.txnIndex.file)
assertEquals(new File(removedLog.dir, logName), removedSegment.log.file)
assertEquals(new File(removedLog.dir, indexName), removedSegment.offsetIndexFile)
assertEquals(new File(removedLog.dir, timeIndexName), removedSegment.timeIndexFile)
assertEquals(new File(removedLog.dir, txnIndexName), removedSegment.txnIndex.file)
// Try to detect the case where a new index type was added and we forgot to update the pointer
// This will only catch cases where the index file is created eagerly instead of lazily
indexFilesOnDiskBeforeDelete.foreach { fileBeforeDelete =>
val fileInIndex = indexFilesAfterDelete.find(_.getName == fileBeforeDelete.getName)
assertEquals(Some(fileBeforeDelete.getName), fileInIndex.map(_.getName),
s"Could not find index file ${fileBeforeDelete.getName} in indexFilesAfterDelete")
assertNotEquals("File reference was not updated in index", fileBeforeDelete.getAbsolutePath,
fileInIndex.get.getAbsolutePath)
}
time.sleep(logManager.initialTaskDelayMs)
assertTrue(logManager.hasLogsToBeDeleted, "Logs deleted too early")
time.sleep(logManager.currentDefaultConfig.fileDeleteDelayMs - logManager.initialTaskDelayMs)
assertFalse(logManager.hasLogsToBeDeleted, "Logs not deleted")
}
@Test
def testCreateAndDeleteOverlyLongTopic(): Unit = {
val invalidTopicName = String.join("", Collections.nCopies(253, "x"))
logManager.getOrCreateLog(new TopicPartition(invalidTopicName, 0), topicId = None)
logManager.asyncDelete(new TopicPartition(invalidTopicName, 0))
}
@Test
def testCheckpointForOnlyAffectedLogs(): Unit = {
val tps = Seq(
new TopicPartition("test-a", 0),
new TopicPartition("test-a", 1),
new TopicPartition("test-a", 2),
new TopicPartition("test-b", 0),
new TopicPartition("test-b", 1))
val allLogs = tps.map(logManager.getOrCreateLog(_, topicId = None))
allLogs.foreach { log =>
for (_ <- 0 until 50)
log.appendAsLeader(TestUtils.singletonRecords("test".getBytes), leaderEpoch = 0)
log.flush(false)
}
logManager.checkpointRecoveryOffsetsInDir(logDir)
val checkpoints = new OffsetCheckpointFile(new File(logDir, LogManager.RecoveryPointCheckpointFile)).read()
tps.zip(allLogs).foreach { case (tp, log) =>
assertEquals(checkpoints(tp), log.recoveryPoint,
"Recovery point should equal checkpoint")
}
}
private def readLog(log: UnifiedLog, offset: Long, maxLength: Int = 1024): FetchDataInfo = {
log.read(offset, maxLength, isolation = FetchIsolation.LOG_END, minOneMessage = true)
}
/**
* Test when a configuration of a topic is updated while its log is getting initialized,
* the config is refreshed when log initialization is finished.
*/
@Test
def testTopicConfigChangeUpdatesLogConfig(): Unit = {
logManager.shutdown()
val spyConfigRepository = spy(new MockConfigRepository)
logManager = createLogManager(configRepository = spyConfigRepository)
val spyLogManager = spy(logManager)
val mockLog = mock(classOf[UnifiedLog])
val testTopicOne = "test-topic-one"
val testTopicTwo = "test-topic-two"
val testTopicOnePartition = new TopicPartition(testTopicOne, 1)
val testTopicTwoPartition = new TopicPartition(testTopicTwo, 1)
spyLogManager.initializingLog(testTopicOnePartition)
spyLogManager.initializingLog(testTopicTwoPartition)
spyLogManager.topicConfigUpdated(testTopicOne)
spyLogManager.finishedInitializingLog(testTopicOnePartition, Some(mockLog))
spyLogManager.finishedInitializingLog(testTopicTwoPartition, Some(mockLog))
// testTopicOne configs loaded again due to the update
verify(spyLogManager).initializingLog(ArgumentMatchers.eq(testTopicOnePartition))
verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(testTopicOnePartition), ArgumentMatchers.any())
verify(spyConfigRepository, times(1)).topicConfig(testTopicOne)
// testTopicTwo configs not loaded again since there was no update
verify(spyLogManager).initializingLog(ArgumentMatchers.eq(testTopicTwoPartition))
verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(testTopicTwoPartition), ArgumentMatchers.any())
verify(spyConfigRepository, never).topicConfig(testTopicTwo)
}
/**
* Test if an error occurs when creating log, log manager removes corresponding
* topic partition from the list of initializing partitions and no configs are retrieved.
*/
@Test
def testConfigChangeGetsCleanedUp(): Unit = {
logManager.shutdown()
val spyConfigRepository = spy(new MockConfigRepository)
logManager = createLogManager(configRepository = spyConfigRepository)
val spyLogManager = spy(logManager)
val testTopicPartition = new TopicPartition("test-topic", 1)
spyLogManager.initializingLog(testTopicPartition)
spyLogManager.finishedInitializingLog(testTopicPartition, None)
assertTrue(logManager.partitionsInitializing.isEmpty)
verify(spyConfigRepository, never).topicConfig(testTopicPartition.topic)
}
/**
* Test when a broker configuration change happens all logs in process of initialization
* pick up latest config when finished with initialization.
*/
@Test
def testBrokerConfigChangeDeliveredToAllLogs(): Unit = {
logManager.shutdown()
val spyConfigRepository = spy(new MockConfigRepository)
logManager = createLogManager(configRepository = spyConfigRepository)
val spyLogManager = spy(logManager)
val mockLog = mock(classOf[UnifiedLog])
val testTopicOne = "test-topic-one"
val testTopicTwo = "test-topic-two"
val testTopicOnePartition = new TopicPartition(testTopicOne, 1)
val testTopicTwoPartition = new TopicPartition(testTopicTwo, 1)
spyLogManager.initializingLog(testTopicOnePartition)
spyLogManager.initializingLog(testTopicTwoPartition)
spyLogManager.brokerConfigUpdated()
spyLogManager.finishedInitializingLog(testTopicOnePartition, Some(mockLog))
spyLogManager.finishedInitializingLog(testTopicTwoPartition, Some(mockLog))
verify(spyConfigRepository, times(1)).topicConfig(testTopicOne)
verify(spyConfigRepository, times(1)).topicConfig(testTopicTwo)
}
/**
* Test when compact is removed that cleaning of the partitions is aborted.
*/
@Test
def testTopicConfigChangeStopCleaningIfCompactIsRemoved(): Unit = {
logManager.shutdown()
logManager = createLogManager(configRepository = new MockConfigRepository)
val spyLogManager = spy(logManager)
val topic = "topic"
val tp0 = new TopicPartition(topic, 0)
val tp1 = new TopicPartition(topic, 1)
val oldProperties = new Properties()
oldProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
val oldLogConfig = LogConfig.fromProps(logConfig.originals, oldProperties)
val log0 = spyLogManager.getOrCreateLog(tp0, topicId = None)
log0.updateConfig(oldLogConfig)
val log1 = spyLogManager.getOrCreateLog(tp1, topicId = None)
log1.updateConfig(oldLogConfig)
assertEquals(Set(log0, log1), spyLogManager.logsByTopic(topic).toSet)
val newProperties = new Properties()
newProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE)
spyLogManager.updateTopicConfig(topic, newProperties, isRemoteLogStorageSystemEnabled = false)
assertTrue(log0.config.delete)
assertTrue(log1.config.delete)
assertFalse(log0.config.compact)
assertFalse(log1.config.compact)
verify(spyLogManager, times(1)).topicConfigUpdated(topic)
verify(spyLogManager, times(1)).abortCleaning(tp0)
verify(spyLogManager, times(1)).abortCleaning(tp1)
}
/**
* Test even if no log is getting initialized, if config change events are delivered
* things continue to work correctly. This test should not throw.
*
* This makes sure that events can be delivered even when no log is getting initialized.
*/
@Test
def testConfigChangesWithNoLogGettingInitialized(): Unit = {
logManager.brokerConfigUpdated()
logManager.topicConfigUpdated("test-topic")
assertTrue(logManager.partitionsInitializing.isEmpty)
}
private def appendRecordsToLog(time: MockTime, parentLogDir: File, partitionId: Int, brokerTopicStats: BrokerTopicStats, expectedSegmentsPerLog: Int): Unit = {
def createRecord = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
val tpFile = new File(parentLogDir, s"$name-$partitionId")
val segmentBytes = 1024
val log = LogTestUtils.createLog(tpFile, logConfig, brokerTopicStats, time.scheduler, time, 0, 0,
5 * 60 * 1000, new ProducerStateManagerConfig(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false), TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT)
assertTrue(expectedSegmentsPerLog > 0)
// calculate numMessages to append to logs. It'll create "expectedSegmentsPerLog" log segments with segment.bytes=1024
val numMessages = Math.floor(segmentBytes * expectedSegmentsPerLog / createRecord.sizeInBytes).asInstanceOf[Int]
try {
for (_ <- 0 until numMessages) {
log.appendAsLeader(createRecord, leaderEpoch = 0)
}
assertEquals(expectedSegmentsPerLog, log.numberOfSegments)
} finally {
log.close()
}
}
private def verifyRemainingLogsToRecoverMetric(spyLogManager: LogManager, expectedParams: Map[String, Int]): Unit = {
val logManagerClassName = classOf[LogManager].getSimpleName
// get all `remainingLogsToRecover` metrics
val logMetrics: ArrayBuffer[Gauge[Int]] = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala
.filter { case (metric, _) => metric.getType == s"$logManagerClassName" && metric.getName == "remainingLogsToRecover" }
.map { case (_, gauge) => gauge }
.asInstanceOf[ArrayBuffer[Gauge[Int]]]
assertEquals(expectedParams.size, logMetrics.size)
val capturedPath: ArgumentCaptor[String] = ArgumentCaptor.forClass(classOf[String])
val expectedCallTimes = expectedParams.values.sum
verify(spyLogManager, times(expectedCallTimes)).decNumRemainingLogs(any[ConcurrentMap[String, Int]], capturedPath.capture())
val paths = capturedPath.getAllValues
expectedParams.foreach {
case (path, totalLogs) =>
// make sure each path is called "totalLogs" times, which means it is decremented to 0 in the end
assertEquals(totalLogs, Collections.frequency(paths, path))
}
// expected the end value is 0
logMetrics.foreach { gauge => assertEquals(0, gauge.value()) }
}
private def verifyRemainingSegmentsToRecoverMetric(spyLogManager: LogManager,
logDirs: Seq[File],
recoveryThreadsPerDataDir: Int,
mockMap: ConcurrentHashMap[String, Int],
expectedParams: Map[String, Int]): Unit = {
val logManagerClassName = classOf[LogManager].getSimpleName
// get all `remainingSegmentsToRecover` metrics
val logSegmentMetrics: ArrayBuffer[Gauge[Int]] = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala
.filter { case (metric, _) => metric.getType == s"$logManagerClassName" && metric.getName == "remainingSegmentsToRecover" }
.map { case (_, gauge) => gauge }
.asInstanceOf[ArrayBuffer[Gauge[Int]]]
// expected each log dir has 1 metrics for each thread
assertEquals(recoveryThreadsPerDataDir * logDirs.size, logSegmentMetrics.size)
val capturedThreadName: ArgumentCaptor[String] = ArgumentCaptor.forClass(classOf[String])
val capturedNumRemainingSegments: ArgumentCaptor[Int] = ArgumentCaptor.forClass(classOf[Int])
// Since we'll update numRemainingSegments from totalSegments to 0 for each thread, so we need to add 1 here
val expectedCallTimes = expectedParams.values.map( num => num + 1 ).sum
verify(mockMap, times(expectedCallTimes)).put(capturedThreadName.capture(), capturedNumRemainingSegments.capture())
// expected the end value is 0
logSegmentMetrics.foreach { gauge => assertEquals(0, gauge.value()) }
val threadNames = capturedThreadName.getAllValues
val numRemainingSegments = capturedNumRemainingSegments.getAllValues
expectedParams.foreach {
case (threadName, totalSegments) =>
// make sure we update the numRemainingSegments from totalSegments to 0 in order for each thread
var expectedCurRemainingSegments = totalSegments + 1
for (i <- 0 until threadNames.size) {
if (threadNames.get(i).contains(threadName)) {
expectedCurRemainingSegments -= 1
assertEquals(expectedCurRemainingSegments, numRemainingSegments.get(i))
}
}
assertEquals(0, expectedCurRemainingSegments)
}
}
private def verifyLogRecoverMetricsRemoved(spyLogManager: LogManager): Unit = {
val spyLogManagerClassName = spyLogManager.getClass.getSimpleName
// get all `remainingLogsToRecover` metrics
def logMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala
.filter { metric => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingLogsToRecover" }
assertTrue(logMetrics.isEmpty)
// get all `remainingSegmentsToRecover` metrics
val logSegmentMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala
.filter { metric => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingSegmentsToRecover" }
assertTrue(logSegmentMetrics.isEmpty)
}
@Test
def testLogRecoveryMetrics(): Unit = {
logManager.shutdown()
val logDir1 = TestUtils.tempDir()
val logDir2 = TestUtils.tempDir()
val logDirs = Seq(logDir1, logDir2)
val recoveryThreadsPerDataDir = 2
// create logManager with expected recovery thread number
logManager = createLogManager(logDirs, recoveryThreadsPerDataDir = recoveryThreadsPerDataDir)
val spyLogManager = spy(logManager)
assertEquals(2, spyLogManager.liveLogDirs.size)
val mockTime = new MockTime()
val mockMap = mock(classOf[ConcurrentHashMap[String, Int]])
val mockBrokerTopicStats = mock(classOf[BrokerTopicStats])
val expectedSegmentsPerLog = 2
// create log segments for log recovery in each log dir
appendRecordsToLog(mockTime, logDir1, 0, mockBrokerTopicStats, expectedSegmentsPerLog)
appendRecordsToLog(mockTime, logDir2, 1, mockBrokerTopicStats, expectedSegmentsPerLog)
// intercept loadLog method to pass expected parameter to do log recovery
doAnswer { invocation =>
val dir: File = invocation.getArgument(0)
val topicConfigOverrides: mutable.Map[String, LogConfig] = invocation.getArgument(5)
val topicPartition = UnifiedLog.parseTopicPartitionName(dir)
val config = topicConfigOverrides.getOrElse(topicPartition.topic, logConfig)
UnifiedLog(
dir = dir,
config = config,
logStartOffset = 0,
recoveryPoint = 0,
maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(5 * 60 * 1000, false),
producerIdExpirationCheckIntervalMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
scheduler = mockTime.scheduler,
time = mockTime,
brokerTopicStats = mockBrokerTopicStats,
logDirFailureChannel = mock(classOf[LogDirFailureChannel]),
// not clean shutdown
lastShutdownClean = false,
topicId = None,
keepPartitionMetadataFile = false,
// pass mock map for verification later
numRemainingSegments = mockMap)
}.when(spyLogManager).loadLog(any[File], any[Boolean], any[Map[TopicPartition, Long]], any[Map[TopicPartition, Long]],
any[LogConfig], any[Map[String, LogConfig]], any[ConcurrentMap[String, Int]], any[UnifiedLog => Boolean]())
// do nothing for removeLogRecoveryMetrics for metrics verification
doNothing().when(spyLogManager).removeLogRecoveryMetrics()
// start the logManager to do log recovery
spyLogManager.startup(Set.empty)
// make sure log recovery metrics are added and removed
verify(spyLogManager, times(1)).addLogRecoveryMetrics(any[ConcurrentMap[String, Int]], any[ConcurrentMap[String, Int]])
verify(spyLogManager, times(1)).removeLogRecoveryMetrics()
// expected 1 log in each log dir since we created 2 partitions with 2 log dirs
val expectedRemainingLogsParams = Map[String, Int](logDir1.getAbsolutePath -> 1, logDir2.getAbsolutePath -> 1)
verifyRemainingLogsToRecoverMetric(spyLogManager, expectedRemainingLogsParams)
val expectedRemainingSegmentsParams = Map[String, Int](
logDir1.getAbsolutePath -> expectedSegmentsPerLog, logDir2.getAbsolutePath -> expectedSegmentsPerLog)
verifyRemainingSegmentsToRecoverMetric(spyLogManager, logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams)
}
@Test
def testLogRecoveryMetricsShouldBeRemovedAfterLogRecovered(): Unit = {
logManager.shutdown()