forked from apache/kafka
/
PartitionTest.scala
3586 lines (3088 loc) · 140 KB
/
PartitionTest.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.cluster
import java.net.InetAddress
import com.yammer.metrics.core.Metric
import kafka.common.UnexpectedAppendOffsetException
import kafka.log._
import kafka.server._
import kafka.server.checkpoints.OffsetCheckpoints
import kafka.utils._
import kafka.zk.KafkaZkClient
import org.apache.kafka.common.errors.{ApiException, FencedLeaderEpochException, InconsistentTopicIdException, NotLeaderOrFollowerException, OffsetNotAvailableException, OffsetOutOfRangeException, UnknownLeaderEpochException}
import org.apache.kafka.common.message.{AlterPartitionResponseData, FetchResponseData}
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{AlterPartitionResponse, FetchRequest, ListOffsetsRequest, RequestHeader}
import org.apache.kafka.common.utils.SystemTime
import org.apache.kafka.common.{InvalidRecordException, IsolationLevel, TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderRecoveryState
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.mockito.ArgumentMatchers
import org.mockito.ArgumentMatchers.{any, anyString}
import org.mockito.Mockito._
import org.mockito.invocation.InvocationOnMock
import java.nio.ByteBuffer
import java.util.Optional
import java.util.concurrent.{CountDownLatch, Semaphore}
import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache}
import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.replica.ClientMetadata
import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, EpochEntry, FetchIsolation, FetchParams, LogAppendInfo, LogDirFailureChannel, LogReadInfo, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
object PartitionTest {
class MockPartitionListener extends PartitionListener {
private var highWatermark: Long = -1L
private var failed: Boolean = false
private var deleted: Boolean = false
override def onHighWatermarkUpdated(partition: TopicPartition, offset: Long): Unit = {
highWatermark = offset
}
override def onFailed(partition: TopicPartition): Unit = {
failed = true
}
override def onDeleted(partition: TopicPartition): Unit = {
deleted = true
}
private def clear(): Unit = {
highWatermark = -1L
failed = false
deleted = false
}
/**
* Verifies the callbacks that have been triggered since the last
* verification. Values different than `-1` are the ones that have
* been updated.
*/
def verify(
expectedHighWatermark: Long = -1L,
expectedFailed: Boolean = false,
expectedDeleted: Boolean = false
): Unit = {
assertEquals(expectedHighWatermark, highWatermark,
"Unexpected high watermark")
assertEquals(expectedFailed, failed,
"Unexpected failed")
assertEquals(expectedDeleted, deleted,
"Unexpected deleted")
clear()
}
}
def followerFetchParams(
replicaId: Int,
replicaEpoch: Long = 1L,
maxWaitMs: Long = 0L,
minBytes: Int = 1,
maxBytes: Int = Int.MaxValue
): FetchParams = {
new FetchParams(
ApiKeys.FETCH.latestVersion,
replicaId,
replicaEpoch,
maxWaitMs,
minBytes,
maxBytes,
FetchIsolation.LOG_END,
Optional.empty()
)
}
def consumerFetchParams(
maxWaitMs: Long = 0L,
minBytes: Int = 1,
maxBytes: Int = Int.MaxValue,
clientMetadata: Option[ClientMetadata] = None,
isolation: FetchIsolation = FetchIsolation.HIGH_WATERMARK
): FetchParams = {
new FetchParams(
ApiKeys.FETCH.latestVersion,
FetchRequest.CONSUMER_REPLICA_ID,
-1,
maxWaitMs,
minBytes,
maxBytes,
isolation,
clientMetadata.asJava
)
}
}
class PartitionTest extends AbstractPartitionTest {
import PartitionTest._
@Test
def testLastFetchedOffsetValidation(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None)
def append(leaderEpoch: Int, count: Int): Unit = {
val recordArray = (1 to count).map { i =>
new SimpleRecord(s"$i".getBytes)
}
val records = MemoryRecords.withRecords(0L, CompressionType.NONE, leaderEpoch,
recordArray: _*)
log.appendAsLeader(records, leaderEpoch = leaderEpoch)
}
append(leaderEpoch = 0, count = 2) // 0
append(leaderEpoch = 3, count = 3) // 2
append(leaderEpoch = 3, count = 3) // 5
append(leaderEpoch = 4, count = 5) // 8
append(leaderEpoch = 7, count = 1) // 13
append(leaderEpoch = 9, count = 3) // 14
assertEquals(17L, log.logEndOffset)
val leaderEpoch = 10
val logStartOffset = 0L
val partition = setupPartitionWithMocks(leaderEpoch = leaderEpoch, isLeader = true)
def epochEndOffset(epoch: Int, endOffset: Long): FetchResponseData.EpochEndOffset = {
new FetchResponseData.EpochEndOffset()
.setEpoch(epoch)
.setEndOffset(endOffset)
}
def read(lastFetchedEpoch: Int, fetchOffset: Long): LogReadInfo = {
fetchFollower(
partition,
remoteReplicaId,
fetchOffset,
logStartOffset,
leaderEpoch = Some(leaderEpoch),
lastFetchedEpoch = Some(lastFetchedEpoch)
)
}
def assertDivergence(
divergingEpoch: FetchResponseData.EpochEndOffset,
readInfo: LogReadInfo
): Unit = {
assertEquals(Optional.of(divergingEpoch), readInfo.divergingEpoch)
assertEquals(0, readInfo.fetchedData.records.sizeInBytes)
}
def assertNoDivergence(readInfo: LogReadInfo): Unit = {
assertEquals(Optional.empty(), readInfo.divergingEpoch)
}
assertDivergence(epochEndOffset(epoch = 0, endOffset = 2), read(lastFetchedEpoch = 2, fetchOffset = 5))
assertDivergence(epochEndOffset(epoch = 0, endOffset= 2), read(lastFetchedEpoch = 0, fetchOffset = 4))
assertDivergence(epochEndOffset(epoch = 4, endOffset = 13), read(lastFetchedEpoch = 6, fetchOffset = 6))
assertDivergence(epochEndOffset(epoch = 4, endOffset = 13), read(lastFetchedEpoch = 5, fetchOffset = 9))
assertDivergence(epochEndOffset(epoch = 10, endOffset = 17), read(lastFetchedEpoch = 10, fetchOffset = 18))
assertNoDivergence(read(lastFetchedEpoch = 0, fetchOffset = 2))
assertNoDivergence(read(lastFetchedEpoch = 7, fetchOffset = 14))
assertNoDivergence(read(lastFetchedEpoch = 9, fetchOffset = 17))
assertNoDivergence(read(lastFetchedEpoch = 10, fetchOffset = 17))
// Reads from epochs larger than we know about should cause an out of range error
assertThrows(classOf[OffsetOutOfRangeException], () => read(lastFetchedEpoch = 11, fetchOffset = 5))
// Move log start offset to the middle of epoch 3
log.updateHighWatermark(log.logEndOffset)
log.maybeIncrementLogStartOffset(newLogStartOffset = 5L, LogStartOffsetIncrementReason.ClientRecordDeletion)
assertDivergence(epochEndOffset(epoch = 2, endOffset = 5), read(lastFetchedEpoch = 2, fetchOffset = 8))
assertNoDivergence(read(lastFetchedEpoch = 0, fetchOffset = 5))
assertNoDivergence(read(lastFetchedEpoch = 3, fetchOffset = 5))
assertThrows(classOf[OffsetOutOfRangeException], () => read(lastFetchedEpoch = 0, fetchOffset = 0))
// Fetch offset lower than start offset should throw OffsetOutOfRangeException
log.maybeIncrementLogStartOffset(newLogStartOffset = 10, LogStartOffsetIncrementReason.ClientRecordDeletion)
assertThrows(classOf[OffsetOutOfRangeException], () => read(lastFetchedEpoch = 5, fetchOffset = 6)) // diverging
assertThrows(classOf[OffsetOutOfRangeException], () => read(lastFetchedEpoch = 3, fetchOffset = 6)) // not diverging
}
@Test
def testMakeLeaderUpdatesEpochCache(): Unit = {
val leaderEpoch = 8
val log = logManager.getOrCreateLog(topicPartition, topicId = None)
log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes)
), leaderEpoch = 0)
log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 5,
new SimpleRecord("k3".getBytes, "v3".getBytes),
new SimpleRecord("k4".getBytes, "v4".getBytes)
), leaderEpoch = 5)
assertEquals(4, log.logEndOffset)
val partition = setupPartitionWithMocks(leaderEpoch = leaderEpoch, isLeader = true)
assertEquals(Some(4), partition.leaderLogIfLocal.map(_.logEndOffset))
val epochEndOffset = partition.lastOffsetForLeaderEpoch(currentLeaderEpoch = Optional.of[Integer](leaderEpoch),
leaderEpoch = leaderEpoch, fetchOnlyFromLeader = true)
assertEquals(4, epochEndOffset.endOffset)
assertEquals(leaderEpoch, epochEndOffset.leaderEpoch)
}
// Verify that partition.removeFutureLocalReplica() and partition.maybeReplaceCurrentWithFutureReplica() can run concurrently
@Test
def testMaybeReplaceCurrentWithFutureReplica(): Unit = {
val latch = new CountDownLatch(1)
logManager.maybeUpdatePreferredLogDir(topicPartition, logDir1.getAbsolutePath)
partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, None)
logManager.maybeUpdatePreferredLogDir(topicPartition, logDir2.getAbsolutePath)
partition.maybeCreateFutureReplica(logDir2.getAbsolutePath, offsetCheckpoints)
val thread1 = new Thread {
override def run(): Unit = {
latch.await()
partition.removeFutureLocalReplica()
}
}
val thread2 = new Thread {
override def run(): Unit = {
latch.await()
partition.maybeReplaceCurrentWithFutureReplica()
}
}
thread1.start()
thread2.start()
latch.countDown()
thread1.join()
thread2.join()
assertEquals(None, partition.futureLog)
}
@Test
def testReplicaFetchToFollower(): Unit = {
val controllerEpoch = 3
val followerId = brokerId + 1
val leaderId = brokerId + 2
val replicas = List[Integer](brokerId, followerId, leaderId).asJava
val isr = List[Integer](brokerId, followerId, leaderId).asJava
val leaderEpoch = 8
val partitionEpoch = 1
assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leaderId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setPartitionEpoch(partitionEpoch)
.setReplicas(replicas)
.setIsNew(true),
offsetCheckpoints, None
))
def assertFetchFromReplicaFails[T <: ApiException](
expectedExceptionClass: Class[T],
leaderEpoch: Option[Int]
): Unit = {
assertThrows(expectedExceptionClass, () => {
fetchFollower(
partition,
replicaId = followerId,
fetchOffset = 0L,
leaderEpoch = leaderEpoch
)
})
}
assertFetchFromReplicaFails(classOf[NotLeaderOrFollowerException], None)
assertFetchFromReplicaFails(classOf[NotLeaderOrFollowerException], Some(leaderEpoch))
assertFetchFromReplicaFails(classOf[UnknownLeaderEpochException], Some(leaderEpoch + 1))
assertFetchFromReplicaFails(classOf[FencedLeaderEpochException], Some(leaderEpoch - 1))
}
@Test
def testFetchFromUnrecognizedFollower(): Unit = {
val controllerEpoch = 3
val leader = brokerId
val validReplica = brokerId + 1
val addingReplica1 = brokerId + 2
val addingReplica2 = brokerId + 3
val replicas = List(leader, validReplica)
val isr = List[Integer](leader, validReplica).asJava
val leaderEpoch = 8
val partitionEpoch = 1
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setPartitionEpoch(partitionEpoch)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true),
offsetCheckpoints, None
))
assertThrows(classOf[UnknownLeaderEpochException], () => {
fetchFollower(
partition,
replicaId = addingReplica1,
fetchOffset = 0L,
leaderEpoch = Some(leaderEpoch)
)
})
assertEquals(None, partition.getReplica(addingReplica1).map(_.stateSnapshot.logEndOffset))
assertThrows(classOf[NotLeaderOrFollowerException], () => {
fetchFollower(
partition,
replicaId = addingReplica2,
fetchOffset = 0L,
leaderEpoch = None
)
})
assertEquals(None, partition.getReplica(addingReplica2).map(_.stateSnapshot.logEndOffset))
// The replicas are added as part of a reassignment
val newReplicas = List(leader, validReplica, addingReplica1, addingReplica2)
val newPartitionEpoch = partitionEpoch + 1
val addingReplicas = List(addingReplica1, addingReplica2)
assertFalse(partition.makeLeader(new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setPartitionEpoch(newPartitionEpoch)
.setReplicas(newReplicas.map(Int.box).asJava)
.setAddingReplicas(addingReplicas.map(Int.box).asJava)
.setIsNew(true),
offsetCheckpoints, None
))
// Now the fetches are allowed
assertEquals(0L, fetchFollower(
partition,
replicaId = addingReplica1,
fetchOffset = 0L,
leaderEpoch = Some(leaderEpoch)
).logEndOffset)
assertEquals(Some(0L), partition.getReplica(addingReplica1).map(_.stateSnapshot.logEndOffset))
assertEquals(0L, fetchFollower(
partition,
replicaId = addingReplica2,
fetchOffset = 0L,
leaderEpoch = None
).logEndOffset)
assertEquals(Some(0L), partition.getReplica(addingReplica2).map(_.stateSnapshot.logEndOffset))
}
// Verify that partition.makeFollower() and partition.appendRecordsToFollowerOrFutureReplica() can run concurrently
@Test
def testMakeFollowerWithWithFollowerAppendRecords(): Unit = {
val appendSemaphore = new Semaphore(0)
val mockTime = new MockTime()
partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
interBrokerProtocolVersion = MetadataVersion.latest,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
alterPartitionListener,
delayedOperations,
metadataCache,
logManager,
alterPartitionManager) {
override def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints, topicId: Option[Uuid]): UnifiedLog = {
val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None)
val logDirFailureChannel = new LogDirFailureChannel(1)
val segments = new LogSegments(log.topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "")
val maxTransactionTimeoutMs = 5 * 60 * 1000
val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, true)
val producerStateManager = new ProducerStateManager(
log.topicPartition,
log.dir,
maxTransactionTimeoutMs,
producerStateManagerConfig,
mockTime
)
val offsets = new LogLoader(
log.dir,
log.topicPartition,
log.config,
mockTime.scheduler,
mockTime,
logDirFailureChannel,
hadCleanShutdown = true,
segments = segments,
logStartOffsetCheckpoint = 0L,
recoveryPointCheckpoint = 0L,
leaderEpochCache,
producerStateManager
).load()
val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition,
logDirFailureChannel)
new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore)
}
}
partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, None)
val appendThread = new Thread {
override def run(): Unit = {
val records = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes)),
baseOffset = 0)
partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false)
}
}
appendThread.start()
TestUtils.waitUntilTrue(() => appendSemaphore.hasQueuedThreads, "follower log append is not called.")
val partitionState = new LeaderAndIsrPartitionState()
.setControllerEpoch(0)
.setLeader(2)
.setLeaderEpoch(1)
.setIsr(List[Integer](0, 1, 2, brokerId).asJava)
.setPartitionEpoch(1)
.setReplicas(List[Integer](0, 1, 2, brokerId).asJava)
.setIsNew(false)
assertTrue(partition.makeFollower(partitionState, offsetCheckpoints, None))
appendSemaphore.release()
appendThread.join()
assertEquals(2L, partition.localLogOrException.logEndOffset)
assertEquals(2L, partition.leaderReplicaIdOpt.get)
}
@Test
// Verify that replacement works when the replicas have the same log end offset but different base offsets in the
// active segment
def testMaybeReplaceCurrentWithFutureReplicaDifferentBaseOffsets(): Unit = {
logManager.maybeUpdatePreferredLogDir(topicPartition, logDir1.getAbsolutePath)
partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, None)
logManager.maybeUpdatePreferredLogDir(topicPartition, logDir2.getAbsolutePath)
partition.maybeCreateFutureReplica(logDir2.getAbsolutePath, offsetCheckpoints)
// Write records with duplicate keys to current replica and roll at offset 6
val currentLog = partition.log.get
currentLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k1".getBytes, "v2".getBytes),
new SimpleRecord("k1".getBytes, "v3".getBytes),
new SimpleRecord("k2".getBytes, "v4".getBytes),
new SimpleRecord("k2".getBytes, "v5".getBytes),
new SimpleRecord("k2".getBytes, "v6".getBytes)
), leaderEpoch = 0)
currentLog.roll()
currentLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
new SimpleRecord("k3".getBytes, "v7".getBytes),
new SimpleRecord("k4".getBytes, "v8".getBytes)
), leaderEpoch = 0)
// Write to the future replica as if the log had been compacted, and do not roll the segment
val buffer = ByteBuffer.allocate(1024)
val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, 0)
builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".getBytes))
builder.appendWithOffset(5L, new SimpleRecord("k2".getBytes, "v6".getBytes))
builder.appendWithOffset(6L, new SimpleRecord("k3".getBytes, "v7".getBytes))
builder.appendWithOffset(7L, new SimpleRecord("k4".getBytes, "v8".getBytes))
val futureLog = partition.futureLocalLogOrException
futureLog.appendAsFollower(builder.build())
assertTrue(partition.maybeReplaceCurrentWithFutureReplica())
}
@Test
def testFetchOffsetSnapshotEpochValidationForLeader(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
def assertSnapshotError(expectedError: Errors, currentLeaderEpoch: Optional[Integer]): Unit = {
try {
partition.fetchOffsetSnapshot(currentLeaderEpoch, fetchOnlyFromLeader = true)
assertEquals(Errors.NONE, expectedError)
} catch {
case error: ApiException => assertEquals(expectedError, Errors.forException(error))
}
}
assertSnapshotError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
assertSnapshotError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
assertSnapshotError(Errors.NONE, Optional.of(leaderEpoch))
assertSnapshotError(Errors.NONE, Optional.empty())
}
@Test
def testFetchOffsetSnapshotEpochValidationForFollower(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
def assertSnapshotError(expectedError: Errors,
currentLeaderEpoch: Optional[Integer],
fetchOnlyLeader: Boolean): Unit = {
try {
partition.fetchOffsetSnapshot(currentLeaderEpoch, fetchOnlyFromLeader = fetchOnlyLeader)
assertEquals(Errors.NONE, expectedError)
} catch {
case error: ApiException => assertEquals(expectedError, Errors.forException(error))
}
}
assertSnapshotError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
assertSnapshotError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
assertSnapshotError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
assertSnapshotError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
assertSnapshotError(Errors.NOT_LEADER_OR_FOLLOWER, Optional.of(leaderEpoch), fetchOnlyLeader = true)
assertSnapshotError(Errors.NOT_LEADER_OR_FOLLOWER, Optional.empty(), fetchOnlyLeader = true)
assertSnapshotError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
assertSnapshotError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
}
@Test
def testOffsetForLeaderEpochValidationForLeader(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
def assertLastOffsetForLeaderError(error: Errors, currentLeaderEpochOpt: Optional[Integer]): Unit = {
val endOffset = partition.lastOffsetForLeaderEpoch(currentLeaderEpochOpt, 0,
fetchOnlyFromLeader = true)
assertEquals(error.code, endOffset.errorCode)
}
assertLastOffsetForLeaderError(Errors.NONE, Optional.empty())
assertLastOffsetForLeaderError(Errors.NONE, Optional.of(leaderEpoch))
assertLastOffsetForLeaderError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
assertLastOffsetForLeaderError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
}
@Test
def testOffsetForLeaderEpochValidationForFollower(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
def assertLastOffsetForLeaderError(error: Errors,
currentLeaderEpochOpt: Optional[Integer],
fetchOnlyLeader: Boolean): Unit = {
val endOffset = partition.lastOffsetForLeaderEpoch(currentLeaderEpochOpt, 0,
fetchOnlyFromLeader = fetchOnlyLeader)
assertEquals(error.code, endOffset.errorCode)
}
assertLastOffsetForLeaderError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
assertLastOffsetForLeaderError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
assertLastOffsetForLeaderError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
assertLastOffsetForLeaderError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
assertLastOffsetForLeaderError(Errors.NOT_LEADER_OR_FOLLOWER, Optional.empty(), fetchOnlyLeader = true)
assertLastOffsetForLeaderError(Errors.NOT_LEADER_OR_FOLLOWER, Optional.of(leaderEpoch), fetchOnlyLeader = true)
assertLastOffsetForLeaderError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
assertLastOffsetForLeaderError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
}
@Test
def testLeaderEpochValidationOnLeader(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
def sendFetch(leaderEpoch: Option[Int]): LogReadInfo = {
fetchFollower(
partition,
remoteReplicaId,
fetchOffset = 0L,
leaderEpoch = leaderEpoch
)
}
assertEquals(0L, sendFetch(leaderEpoch = None).logEndOffset)
assertEquals(0L, sendFetch(leaderEpoch = Some(leaderEpoch)).logEndOffset)
assertThrows(classOf[FencedLeaderEpochException], () => sendFetch(Some(leaderEpoch - 1)))
assertThrows(classOf[UnknownLeaderEpochException], () => sendFetch(Some(leaderEpoch + 1)))
}
@Test
def testLeaderEpochValidationOnFollower(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
def sendFetch(
leaderEpoch: Option[Int],
clientMetadata: Option[ClientMetadata]
): LogReadInfo = {
fetchConsumer(
partition,
fetchOffset = 0L,
leaderEpoch = leaderEpoch,
clientMetadata = clientMetadata
)
}
// Follower fetching is only allowed when the client provides metadata
assertThrows(classOf[NotLeaderOrFollowerException], () => sendFetch(None, None))
assertThrows(classOf[NotLeaderOrFollowerException], () => sendFetch(Some(leaderEpoch), None))
assertThrows(classOf[FencedLeaderEpochException], () => sendFetch(Some(leaderEpoch - 1), None))
assertThrows(classOf[UnknownLeaderEpochException], () => sendFetch(Some(leaderEpoch + 1), None))
val clientMetadata = new DefaultClientMetadata(
"rack",
"clientId",
InetAddress.getLoopbackAddress,
KafkaPrincipal.ANONYMOUS,
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT).value
)
assertEquals(0L, sendFetch(leaderEpoch = None, Some(clientMetadata)).logEndOffset)
assertEquals(0L, sendFetch(leaderEpoch = Some(leaderEpoch), Some(clientMetadata)).logEndOffset)
assertThrows(classOf[FencedLeaderEpochException], () => sendFetch(Some(leaderEpoch - 1), Some(clientMetadata)))
assertThrows(classOf[UnknownLeaderEpochException], () => sendFetch(Some(leaderEpoch + 1), Some(clientMetadata)))
}
@Test
def testFetchOffsetForTimestampEpochValidationForLeader(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
def assertFetchOffsetError(error: Errors,
currentLeaderEpochOpt: Optional[Integer]): Unit = {
try {
partition.fetchOffsetForTimestamp(0L,
isolationLevel = None,
currentLeaderEpoch = currentLeaderEpochOpt,
fetchOnlyFromLeader = true)
if (error != Errors.NONE)
fail(s"Expected readRecords to fail with error $error")
} catch {
case e: Exception =>
assertEquals(error, Errors.forException(e))
}
}
assertFetchOffsetError(Errors.NONE, Optional.empty())
assertFetchOffsetError(Errors.NONE, Optional.of(leaderEpoch))
assertFetchOffsetError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1))
assertFetchOffsetError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1))
}
@Test
def testFetchOffsetForTimestampEpochValidationForFollower(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
def assertFetchOffsetError(error: Errors,
currentLeaderEpochOpt: Optional[Integer],
fetchOnlyLeader: Boolean): Unit = {
try {
partition.fetchOffsetForTimestamp(0L,
isolationLevel = None,
currentLeaderEpoch = currentLeaderEpochOpt,
fetchOnlyFromLeader = fetchOnlyLeader)
if (error != Errors.NONE)
fail(s"Expected readRecords to fail with error $error")
} catch {
case e: Exception =>
assertEquals(error, Errors.forException(e))
}
}
assertFetchOffsetError(Errors.NONE, Optional.empty(), fetchOnlyLeader = false)
assertFetchOffsetError(Errors.NONE, Optional.of(leaderEpoch), fetchOnlyLeader = false)
assertFetchOffsetError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = false)
assertFetchOffsetError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = false)
assertFetchOffsetError(Errors.NOT_LEADER_OR_FOLLOWER, Optional.empty(), fetchOnlyLeader = true)
assertFetchOffsetError(Errors.NOT_LEADER_OR_FOLLOWER, Optional.of(leaderEpoch), fetchOnlyLeader = true)
assertFetchOffsetError(Errors.FENCED_LEADER_EPOCH, Optional.of(leaderEpoch - 1), fetchOnlyLeader = true)
assertFetchOffsetError(Errors.UNKNOWN_LEADER_EPOCH, Optional.of(leaderEpoch + 1), fetchOnlyLeader = true)
}
@Test
def testFetchLatestOffsetIncludesLeaderEpoch(): Unit = {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true)
val timestampAndOffsetOpt = partition.fetchOffsetForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP,
isolationLevel = None,
currentLeaderEpoch = Optional.empty(),
fetchOnlyFromLeader = true)
assertTrue(timestampAndOffsetOpt.isDefined)
val timestampAndOffset = timestampAndOffsetOpt.get
assertEquals(leaderEpoch, timestampAndOffset.leaderEpoch.get)
}
/**
* This test checks that after a new leader election, we don't answer any ListOffsetsRequest until
* the HW of the new leader has caught up to its startLogOffset for this epoch. From a client
* perspective this helps guarantee monotonic offsets
*
* @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-207%3A+Offsets+returned+by+ListOffsetsResponse+should+be+monotonically+increasing+even+during+a+partition+leader+change">KIP-207</a>
*/
@Test
def testMonotonicOffsetsAfterLeaderChange(): Unit = {
val controllerEpoch = 3
val leader = brokerId
val follower1 = brokerId + 1
val follower2 = brokerId + 2
val replicas = List(leader, follower1, follower2)
val isr = List[Integer](leader, follower2).asJava
val leaderEpoch = 8
val batch1 = TestUtils.records(records = List(
new SimpleRecord(10, "k1".getBytes, "v1".getBytes),
new SimpleRecord(11,"k2".getBytes, "v2".getBytes)))
val batch2 = TestUtils.records(records = List(new SimpleRecord("k3".getBytes, "v1".getBytes),
new SimpleRecord(20,"k4".getBytes, "v2".getBytes),
new SimpleRecord(21,"k5".getBytes, "v3".getBytes)))
val leaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setPartitionEpoch(1)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true)
assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, None), "Expected first makeLeader() to return 'leader changed'")
assertEquals(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch")
assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR")
val requestLocal = RequestLocal.withThreadConfinedCaching
// after makeLeader(() call, partition should know about all the replicas
// append records with initial leader epoch
partition.appendRecordsToLeader(batch1, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal)
partition.appendRecordsToLeader(batch2, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal)
assertEquals(partition.localLogOrException.logStartOffset, partition.localLogOrException.highWatermark,
"Expected leader's HW not move")
def fetchOffsetsForTimestamp(timestamp: Long, isolation: Option[IsolationLevel]): Either[ApiException, Option[TimestampAndOffset]] = {
try {
Right(partition.fetchOffsetForTimestamp(
timestamp = timestamp,
isolationLevel = isolation,
currentLeaderEpoch = Optional.of(partition.getLeaderEpoch),
fetchOnlyFromLeader = true
))
} catch {
case e: ApiException => Left(e)
}
}
// let the follower in ISR move leader's HW to move further but below LEO
fetchFollower(partition, replicaId = follower1, fetchOffset = 0L)
fetchFollower(partition, replicaId = follower1, fetchOffset = 2L)
fetchFollower(partition, replicaId = follower2, fetchOffset = 0L)
fetchFollower(partition, replicaId = follower2, fetchOffset = 2L)
// Simulate successful ISR update
alterPartitionManager.completeIsrUpdate(2)
// At this point, the leader has gotten 5 writes, but followers have only fetched two
assertEquals(2, partition.localLogOrException.highWatermark)
// Get the LEO
fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, None) match {
case Right(Some(offsetAndTimestamp)) => assertEquals(5, offsetAndTimestamp.offset)
case Right(None) => fail("Should have seen some offsets")
case Left(e) => fail("Should not have seen an error")
}
// Get the HW
fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) => assertEquals(2, offsetAndTimestamp.offset)
case Right(None) => fail("Should have seen some offsets")
case Left(e) => fail("Should not have seen an error")
}
// Get a offset beyond the HW by timestamp, get a None
assertEquals(Right(None), fetchOffsetsForTimestamp(30, Some(IsolationLevel.READ_UNCOMMITTED)))
// Make into a follower
val followerState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(follower2)
.setLeaderEpoch(leaderEpoch + 1)
.setIsr(isr)
.setPartitionEpoch(4)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(false)
assertTrue(partition.makeFollower(followerState, offsetCheckpoints, None))
// Back to leader, this resets the startLogOffset for this epoch (to 2), we're now in the fault condition
val newLeaderState = new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(leader)
.setLeaderEpoch(leaderEpoch + 2)
.setIsr(isr)
.setPartitionEpoch(5)
.setReplicas(replicas.map(Int.box).asJava)
.setIsNew(false)
assertTrue(partition.makeLeader(newLeaderState, offsetCheckpoints, None))
// Try to get offsets as a client
fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) => fail("Should have failed with OffsetNotAvailable")
case Right(None) => fail("Should have seen an error")
case Left(e: OffsetNotAvailableException) => // ok
case Left(e: ApiException) => fail(s"Expected OffsetNotAvailableException, got $e")
}
// If request is not from a client, we skip the check
fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, None) match {
case Right(Some(offsetAndTimestamp)) => assertEquals(5, offsetAndTimestamp.offset)
case Right(None) => fail("Should have seen some offsets")
case Left(e: ApiException) => fail(s"Got ApiException $e")
}
// If we request the earliest timestamp, we skip the check
fetchOffsetsForTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) => assertEquals(0, offsetAndTimestamp.offset)
case Right(None) => fail("Should have seen some offsets")
case Left(e: ApiException) => fail(s"Got ApiException $e")
}
// If we request the earliest local timestamp, we skip the check
fetchOffsetsForTimestamp(ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) => assertEquals(0, offsetAndTimestamp.offset)
case Right(None) => fail("Should have seen some offsets")
case Left(e: ApiException) => fail(s"Got ApiException $e")
}
// If we request an offset by timestamp earlier than the HW, we are ok
fetchOffsetsForTimestamp(11, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) =>
assertEquals(1, offsetAndTimestamp.offset)
assertEquals(11, offsetAndTimestamp.timestamp)
case Right(None) => fail("Should have seen some offsets")
case Left(e: ApiException) => fail(s"Got ApiException $e")
}
// Request an offset by timestamp beyond the HW, get an error now since we're in a bad state
fetchOffsetsForTimestamp(100, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) => fail("Should have failed")
case Right(None) => fail("Should have failed")
case Left(e: OffsetNotAvailableException) => // ok
case Left(e: ApiException) => fail(s"Should have seen OffsetNotAvailableException, saw $e")
}
// Next fetch from replicas, HW is moved up to 5 (ahead of the LEO)
fetchFollower(partition, replicaId = follower1, fetchOffset = 5L)
fetchFollower(partition, replicaId = follower2, fetchOffset = 5L)
// Simulate successful ISR update
alterPartitionManager.completeIsrUpdate(6)
// Error goes away
fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match {
case Right(Some(offsetAndTimestamp)) => assertEquals(5, offsetAndTimestamp.offset)
case Right(None) => fail("Should have seen some offsets")
case Left(e: ApiException) => fail(s"Got ApiException $e")
}
// Now we see None instead of an error for out of range timestamp
assertEquals(Right(None), fetchOffsetsForTimestamp(100, Some(IsolationLevel.READ_UNCOMMITTED)))
}
@Test
def testAppendRecordsAsFollowerBelowLogStartOffset(): Unit = {
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
val log = partition.localLogOrException
val initialLogStartOffset = 5L
partition.truncateFullyAndStartAt(initialLogStartOffset, isFuture = false)
assertEquals(initialLogStartOffset, log.logEndOffset,
s"Log end offset after truncate fully and start at $initialLogStartOffset:")
assertEquals(initialLogStartOffset, log.logStartOffset,
s"Log start offset after truncate fully and start at $initialLogStartOffset:")
// verify that we cannot append records that do not contain log start offset even if the log is empty
assertThrows(classOf[UnexpectedAppendOffsetException], () =>
// append one record with offset = 3
partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 3L), isFuture = false)
)
assertEquals(initialLogStartOffset, log.logEndOffset,
s"Log end offset should not change after failure to append")
// verify that we can append records that contain log start offset, even when first
// offset < log start offset if the log is empty
val newLogStartOffset = 4L
val records = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes),
new SimpleRecord("k3".getBytes, "v3".getBytes)),
baseOffset = newLogStartOffset)
partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false)
assertEquals(7L, log.logEndOffset, s"Log end offset after append of 3 records with base offset $newLogStartOffset:")
assertEquals(newLogStartOffset, log.logStartOffset, s"Log start offset after append of 3 records with base offset $newLogStartOffset:")
// and we can append more records after that
partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 7L), isFuture = false)
assertEquals(8L, log.logEndOffset, s"Log end offset after append of 1 record at offset 7:")
assertEquals(newLogStartOffset, log.logStartOffset, s"Log start offset not expected to change:")
// but we cannot append to offset < log start if the log is not empty
val records2 = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes)),
baseOffset = 3L)
assertThrows(classOf[UnexpectedAppendOffsetException], () => partition.appendRecordsToFollowerOrFutureReplica(records2, isFuture = false))
assertEquals(8L, log.logEndOffset, s"Log end offset should not change after failure to append")
// we still can append to next offset
partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 8L), isFuture = false)
assertEquals(9L, log.logEndOffset, s"Log end offset after append of 1 record at offset 8:")
assertEquals(newLogStartOffset, log.logStartOffset, s"Log start offset not expected to change:")
}
@Test
def testListOffsetIsolationLevels(): Unit = {
val controllerEpoch = 0
val leaderEpoch = 5
val replicas = List[Integer](brokerId, brokerId + 1).asJava
val isr = replicas
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
assertTrue(partition.makeLeader(new LeaderAndIsrPartitionState()
.setControllerEpoch(controllerEpoch)
.setLeader(brokerId)
.setLeaderEpoch(leaderEpoch)
.setIsr(isr)
.setPartitionEpoch(1)
.setReplicas(replicas)
.setIsNew(true), offsetCheckpoints, None), "Expected become leader transition to succeed")
assertEquals(leaderEpoch, partition.getLeaderEpoch)
val records = createTransactionalRecords(List(
new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes),
new SimpleRecord("k3".getBytes, "v3".getBytes)),
baseOffset = 0L,
producerId = 2L)