-
Notifications
You must be signed in to change notification settings - Fork 15.2k
Expand file tree
/
Copy pathReplicaManager.scala
More file actions
2568 lines (2320 loc) · 131 KB
/
ReplicaManager.scala
File metadata and controls
2568 lines (2320 loc) · 131 KB
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.server
import com.yammer.metrics.core.Meter
import kafka.cluster.Partition
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.ReplicaManager.{AtMinIsrPartitionCountMetricName, FailedIsrUpdatesPerSecMetricName, IsrExpandsPerSecMetricName, IsrShrinksPerSecMetricName, LeaderCountMetricName, OfflineReplicaCountMetricName, PartitionCountMetricName, PartitionsWithLateTransactionsCountMetricName, ProducerIdCountMetricName, ReassigningPartitionsMetricName, UnderMinIsrPartitionCountMetricName, UnderReplicatedPartitionsMetricName, createLogReadResult, isListOffsetsTimestampUnsupported}
import kafka.server.share.DelayedShareFetch
import kafka.utils._
import org.apache.kafka.common.{IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.{Plugin, Topic}
import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult
import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic
import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
import org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse, ListOffsetsTopicResponse}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult}
import org.apache.kafka.common.message.{DescribeLogDirsResponseData, DescribeProducersResponseData}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.internal._
import org.apache.kafka.common.replica.PartitionView.DefaultPartitionView
import org.apache.kafka.common.replica.ReplicaView.DefaultReplicaView
import org.apache.kafka.common.replica._
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.internals.Exit
import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionLogConfig}
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
import org.apache.kafka.logger.StateChangeLogger
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.metadata.MetadataCache
import org.apache.kafka.server.purgatory.DelayedProduce.ProducePartitionStatus
import org.apache.kafka.server.LogAppendResult.LogAppendSummary
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TransactionVersion}
import org.apache.kafka.server.log.remote.TopicPartitionLog
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManager
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.network.BrokerEndPoint
import org.apache.kafka.server.partition.{AlterPartitionManager, PartitionListener}
import org.apache.kafka.server.purgatory.DelayedProduce.PartitionStatusValidator.Result
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedProduce, DelayedRemoteFetch, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey}
import org.apache.kafka.server.quota.{ReplicaQuota, ReplicationQuotaManager}
import org.apache.kafka.server.share.fetch.{DelayedShareFetchKey, DelayedShareFetchPartitionKey}
import org.apache.kafka.server.storage.log.{FetchParams, FetchPartitionData}
import org.apache.kafka.server.transaction.AddPartitionsToTxnManager
import org.apache.kafka.server.transaction.AddPartitionsToTxnManager.TransactionSupportedOperation
import org.apache.kafka.server.util.timer.{SystemTimer, TimerTask}
import org.apache.kafka.server.util.{Scheduler, ShutdownableThread}
import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, HostedPartition, LogAppendResult, LogDeleteRecordsResult, common}
import org.apache.kafka.storage.internals.checkpoint.{LazyOffsetCheckpoints, OffsetCheckpointFile, OffsetCheckpoints}
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, FetchPartitionStatus, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogManager, LogOffsetMetadata, LogReadInfo, LogReadResult, OffsetResultHolder, RecordValidationException, RecordValidationStats, RemoteLogReadResult, RemoteStorageFetchInfo, UnifiedLog, VerificationGuard}
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import java.io.File
import java.lang.{Long => JLong}
import java.nio.file.{Files, Paths}
import java.util
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, Future, RejectedExecutionException, TimeUnit}
import java.util.{Collections, Optional, OptionalInt, OptionalLong}
import java.util.function.Consumer
import scala.collection.{Map, Seq, Set, immutable, mutable}
import scala.jdk.CollectionConverters._
import scala.jdk.FunctionConverters.enrichAsJavaConsumer
import scala.jdk.OptionConverters.RichOptional
object ReplicaManager {
val HighWatermarkFilename = "replication-offset-checkpoint"
private val LeaderCountMetricName = "LeaderCount"
private val PartitionCountMetricName = "PartitionCount"
private val OfflineReplicaCountMetricName = "OfflineReplicaCount"
private val UnderReplicatedPartitionsMetricName = "UnderReplicatedPartitions"
private val UnderMinIsrPartitionCountMetricName = "UnderMinIsrPartitionCount"
private val AtMinIsrPartitionCountMetricName = "AtMinIsrPartitionCount"
private val ReassigningPartitionsMetricName = "ReassigningPartitions"
private val PartitionsWithLateTransactionsCountMetricName = "PartitionsWithLateTransactionsCount"
private val ProducerIdCountMetricName = "ProducerIdCount"
private val IsrExpandsPerSecMetricName = "IsrExpandsPerSec"
private val IsrShrinksPerSecMetricName = "IsrShrinksPerSec"
private val FailedIsrUpdatesPerSecMetricName = "FailedIsrUpdatesPerSec"
private[server] val GaugeMetricNames = Set(
LeaderCountMetricName,
PartitionCountMetricName,
OfflineReplicaCountMetricName,
UnderReplicatedPartitionsMetricName,
UnderMinIsrPartitionCountMetricName,
AtMinIsrPartitionCountMetricName,
ReassigningPartitionsMetricName,
PartitionsWithLateTransactionsCountMetricName,
ProducerIdCountMetricName
)
private[server] val MeterMetricNames = Set(
IsrExpandsPerSecMetricName,
IsrShrinksPerSecMetricName,
FailedIsrUpdatesPerSecMetricName
)
private[server] val MetricNames = GaugeMetricNames.union(MeterMetricNames)
private val timestampMinSupportedVersion: immutable.Map[Long, Short] = immutable.Map[Long, Short](
ListOffsetsRequest.EARLIEST_TIMESTAMP -> 1.toShort,
ListOffsetsRequest.LATEST_TIMESTAMP -> 1.toShort,
ListOffsetsRequest.MAX_TIMESTAMP -> 7.toShort,
ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP -> 8.toShort,
ListOffsetsRequest.LATEST_TIERED_TIMESTAMP -> 9.toShort,
ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP -> 11.toShort
)
def createLogReadResult(highWatermark: Long,
leaderLogStartOffset: Long,
leaderLogEndOffset: Long,
e: Throwable): LogReadResult = {
new LogReadResult(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
Optional.empty(),
highWatermark,
leaderLogStartOffset,
leaderLogEndOffset,
-1L,
-1L,
OptionalLong.empty(),
Errors.forException(e));
}
private[server] def isListOffsetsTimestampUnsupported(timestamp: JLong, version: Short): Boolean = {
timestamp < 0 &&
(!timestampMinSupportedVersion.contains(timestamp) || version < timestampMinSupportedVersion(timestamp))
}
}
class ReplicaManager(val config: KafkaConfig,
metrics: Metrics,
time: Time,
scheduler: Scheduler,
val logManager: LogManager,
val remoteLogManager: Option[RemoteLogManager] = None,
quotaManagers: QuotaManagers,
val metadataCache: MetadataCache,
logDirFailureChannel: LogDirFailureChannel,
val alterPartitionManager: AlterPartitionManager,
val brokerTopicStats: BrokerTopicStats = new BrokerTopicStats(),
delayedProducePurgatoryParam: Option[DelayedOperationPurgatory[DelayedProduce]] = None,
delayedFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedFetch]] = None,
delayedDeleteRecordsPurgatoryParam: Option[DelayedOperationPurgatory[DelayedDeleteRecords]] = None,
delayedRemoteFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedRemoteFetch]] = None,
delayedRemoteListOffsetsPurgatoryParam: Option[DelayedOperationPurgatory[DelayedRemoteListOffsets]] = None,
delayedShareFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedShareFetch]] = None,
val brokerEpochSupplier: () => Long = () => -1,
addPartitionsToTxnManager: Option[AddPartitionsToTxnManager] = None,
val directoryEventHandler: DirectoryEventHandler = DirectoryEventHandler.NOOP,
val defaultActionQueue: ActionQueue = new DelayedActionQueue
) extends Logging {
// Changing the package or class name may cause incompatibility with existing code and metrics configuration
private val metricsPackage = "kafka.server"
private val metricsClassName = "ReplicaManager"
private val metricsGroup = new KafkaMetricsGroup(metricsPackage, metricsClassName)
private val addPartitionsToTxnConfig = new AddPartitionsToTxnConfig(config)
private val shareFetchPurgatoryName = "ShareFetch"
private val delayedShareFetchTimer = new SystemTimer(shareFetchPurgatoryName)
val delayedProducePurgatory = delayedProducePurgatoryParam.getOrElse(
new DelayedOperationPurgatory[DelayedProduce](
"Produce", config.brokerId,
config.producerPurgatoryPurgeIntervalRequests))
val delayedFetchPurgatory = delayedFetchPurgatoryParam.getOrElse(
new DelayedOperationPurgatory[DelayedFetch](
"Fetch", config.brokerId,
config.fetchPurgatoryPurgeIntervalRequests))
val delayedDeleteRecordsPurgatory = delayedDeleteRecordsPurgatoryParam.getOrElse(
new DelayedOperationPurgatory[DelayedDeleteRecords](
"DeleteRecords", config.brokerId,
config.deleteRecordsPurgatoryPurgeIntervalRequests))
// delayedRemoteFetchPurgatory purgeInterval is set to 0 to release the references of completed DelayedRemoteFetch
// instances immediately for GC. The DelayedRemoteFetch instance internally holds the RemoteLogReadResult that can be
// up to the size of `fetch.max.bytes` which defaults to 50 MB.
val delayedRemoteFetchPurgatory = delayedRemoteFetchPurgatoryParam.getOrElse(
new DelayedOperationPurgatory[DelayedRemoteFetch](
"RemoteFetch", config.brokerId, 0))
val delayedRemoteListOffsetsPurgatory = delayedRemoteListOffsetsPurgatoryParam.getOrElse(
new DelayedOperationPurgatory[DelayedRemoteListOffsets](
"RemoteListOffsets", config.brokerId))
val delayedShareFetchPurgatory = delayedShareFetchPurgatoryParam.getOrElse(
new DelayedOperationPurgatory[DelayedShareFetch](
shareFetchPurgatoryName, delayedShareFetchTimer, config.brokerId,
config.shareGroupConfig.shareFetchPurgatoryPurgeIntervalRequests))
/* epoch of the controller that last changed the leader */
protected val localBrokerId = config.brokerId
protected val allPartitions = new ConcurrentHashMap[TopicPartition, HostedPartition[Partition]]
private val replicaStateChangeLock = new Object
val replicaFetcherManager = createReplicaFetcherManager(metrics, time, quotaManagers.follower)
private[server] val replicaAlterLogDirsManager = createReplicaAlterLogDirsManager(quotaManagers.alterLogDirs, brokerTopicStats)
private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
@volatile private[server] var highWatermarkCheckpoints: Map[String, OffsetCheckpointFile] = logManager.liveLogDirs.asScala.map(dir =>
(dir.getAbsolutePath, new OffsetCheckpointFile(new File(dir, ReplicaManager.HighWatermarkFilename), logDirFailureChannel))).toMap
@volatile private var isInControlledShutdown = false
this.logIdent = s"[ReplicaManager broker=$localBrokerId] "
protected val stateChangeLogger = new StateChangeLogger(localBrokerId)
private var logDirFailureHandler: LogDirFailureHandler = _
private class LogDirFailureHandler(name: String) extends ShutdownableThread(name) {
override def doWork(): Unit = {
val newOfflineLogDir = logDirFailureChannel.takeNextOfflineLogDir()
handleLogDirFailure(newOfflineLogDir)
}
}
// Visible for testing
private[server] val replicaSelectorPlugin: Option[Plugin[ReplicaSelector]] = createReplicaSelector(metrics)
metricsGroup.newGauge(LeaderCountMetricName, () => leaderPartitionsIterator.size)
// Visible for testing
private[kafka] val partitionCount = metricsGroup.newGauge(PartitionCountMetricName, () => allPartitions.size)
metricsGroup.newGauge(OfflineReplicaCountMetricName, () => offlinePartitionCount)
metricsGroup.newGauge(UnderReplicatedPartitionsMetricName, () => underReplicatedPartitionCount)
metricsGroup.newGauge(UnderMinIsrPartitionCountMetricName, () => leaderPartitionsIterator.count(_.isUnderMinIsr))
metricsGroup.newGauge(AtMinIsrPartitionCountMetricName, () => leaderPartitionsIterator.count(_.isAtMinIsr))
metricsGroup.newGauge(ReassigningPartitionsMetricName, () => reassigningPartitionsCount)
metricsGroup.newGauge(PartitionsWithLateTransactionsCountMetricName, () => lateTransactionsCount)
metricsGroup.newGauge(ProducerIdCountMetricName, () => producerIdCount)
private def reassigningPartitionsCount: Int = leaderPartitionsIterator.count(_.isReassigning)
private def lateTransactionsCount: Int = {
val currentTimeMs = time.milliseconds()
leaderPartitionsIterator.count(_.hasLateTransaction(currentTimeMs))
}
def producerIdCount: Int = onlinePartitionsIterator.map(_.producerIdCount).sum
val isrExpandRate: Meter = metricsGroup.newMeter(IsrExpandsPerSecMetricName, "expands", TimeUnit.SECONDS)
val isrShrinkRate: Meter = metricsGroup.newMeter(IsrShrinksPerSecMetricName, "shrinks", TimeUnit.SECONDS)
val failedIsrUpdatesRate: Meter = metricsGroup.newMeter(FailedIsrUpdatesPerSecMetricName, "failedUpdates", TimeUnit.SECONDS)
def underReplicatedPartitionCount: Int = leaderPartitionsIterator.count(_.isUnderReplicated)
def startHighWatermarkCheckPointThread(): Unit = {
if (highWatermarkCheckPointThreadStarted.compareAndSet(false, true))
scheduler.schedule("highwatermark-checkpoint", () => checkpointHighWatermarks(), 0L, config.replicaHighWatermarkCheckpointIntervalMs)
}
// When ReplicaAlterDirThread finishes replacing a current replica with a future replica, it will
// remove the partition from the partition state map. But it will not close itself even if the
// partition state map is empty. Thus we need to call shutdownIdleReplicaAlterDirThread() periodically
// to shutdown idle ReplicaAlterDirThread
private def shutdownIdleReplicaAlterLogDirsThread(): Unit = {
replicaAlterLogDirsManager.shutdownIdleFetcherThreads()
}
def resizeFetcherThreadPool(newSize: Int): Unit = {
replicaFetcherManager.resizeThreadPool(newSize)
}
def getLog(topicPartition: TopicPartition): Option[UnifiedLog] = logManager.getLog(topicPartition).toScala
def startup(): Unit = {
// start ISR expiration thread
// A follower can lag behind leader for up to config.replicaLagTimeMaxMs x 1.5 before it is removed from ISR
scheduler.schedule("isr-expiration", () => maybeShrinkIsr(), 0L, config.replicaLagTimeMaxMs / 2)
scheduler.schedule("shutdown-idle-replica-alter-log-dirs-thread", () => shutdownIdleReplicaAlterLogDirsThread(), 0L, 10000L)
logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler")
logDirFailureHandler.start()
addPartitionsToTxnManager.foreach(_.start())
remoteLogManager.foreach(rlm => rlm.setDelayedOperationPurgatory(delayedRemoteListOffsetsPurgatory))
}
private def maybeRemoveTopicMetrics(topic: String): Unit = {
val topicHasNonOfflinePartition = allPartitions.values.asScala.exists {
case online: HostedPartition.Online[Partition] => topic == online.partition.topic
case _: HostedPartition.Offline[Partition] => false
case _: HostedPartition.None[Partition] => false
}
if (!topicHasNonOfflinePartition) // nothing online or deferred
brokerTopicStats.removeMetrics(topic)
}
private def completeDelayedOperationsWhenNotPartitionLeader(topicPartition: TopicPartition, topicId: Option[Uuid]): Unit = {
val topicPartitionOperationKey = new TopicPartitionOperationKey(topicPartition)
delayedProducePurgatory.checkAndComplete(topicPartitionOperationKey)
delayedFetchPurgatory.checkAndComplete(topicPartitionOperationKey)
delayedRemoteFetchPurgatory.checkAndComplete(topicPartitionOperationKey)
delayedRemoteListOffsetsPurgatory.checkAndComplete(topicPartitionOperationKey)
if (topicId.isDefined) delayedShareFetchPurgatory.checkAndComplete(
new DelayedShareFetchPartitionKey(topicId.get, topicPartition.partition()))
}
/**
* Complete any local follower fetches that have been unblocked since new data is available
* from the leader for one or more partitions. Should only be called by ReplicaFetcherThread
* after successfully replicating from the leader.
*/
private[server] def completeDelayedFetchRequests(topicPartitions: Seq[TopicPartition]): Unit = {
topicPartitions.foreach(tp => delayedFetchPurgatory.checkAndComplete(new TopicPartitionOperationKey(tp)))
}
/**
* Complete any delayed share fetch requests that have been unblocked since new data is available from the leader
* for one of the partitions. This could happen due to acknowledgements, acquisition lock timeout of records, partition
* locks getting freed and release of acquired records due to share session close.
* @param delayedShareFetchKey The key corresponding to which the share fetch request has been stored in the purgatory
*/
private[server] def completeDelayedShareFetchRequest(delayedShareFetchKey: DelayedShareFetchKey): Unit = {
delayedShareFetchPurgatory.checkAndComplete(delayedShareFetchKey)
}
/**
* Add and watch a share fetch request in the delayed share fetch purgatory corresponding to a set of keys in case it cannot be
* completed instantaneously, otherwise complete it.
* @param delayedShareFetch Refers to the DelayedOperation over share fetch request
* @param delayedShareFetchKeys The keys corresponding to which the delayed share fetch request will be stored in the purgatory
*/
private[server] def addDelayedShareFetchRequest(delayedShareFetch: DelayedShareFetch,
delayedShareFetchKeys : util.List[DelayedShareFetchKey]): Unit = {
delayedShareFetchPurgatory.tryCompleteElseWatch(delayedShareFetch, delayedShareFetchKeys)
}
/**
* Add a timer task to the delayedShareFetchTimer.
* @param timerTask The timer task to be added to the delayedShareFetchTimer
*/
private[server] def addShareFetchTimerRequest(timerTask: TimerTask): Unit = {
delayedShareFetchTimer.add(timerTask)
}
/**
* Registers the provided listener to the partition iff the partition is online.
*/
def maybeAddListener(partition: TopicPartition, listener: PartitionListener): Boolean = {
getPartition(partition) match {
case online: HostedPartition.Online[Partition] =>
online.partition.maybeAddListener(listener)
case _ =>
false
}
}
/**
* Removes the provided listener from the partition.
*/
def removeListener(partition: TopicPartition, listener: PartitionListener): Unit = {
getPartition(partition) match {
case online: HostedPartition.Online[Partition] =>
online.partition.removeListener(listener)
case _ => // Ignore
}
}
/**
* Stop the given partitions.
*
* @param partitionsToStop set of topic-partitions to be stopped which also indicates whether to remove the
* partition data from the local and remote log storage.
*
* @return A map from partitions to exceptions which occurred.
* If no errors occurred, the map will be empty.
*/
private def stopPartitions(partitionsToStop: Set[StopPartition]): Map[TopicPartition, Throwable] = {
// First stop fetchers for all partitions.
val partitions = partitionsToStop.map(_.topicPartition)
replicaFetcherManager.removeFetcherForPartitions(partitions)
replicaAlterLogDirsManager.removeFetcherForPartitions(partitions)
// Second remove deleted partitions from the partition map. Fetchers rely on the
// ReplicaManager to get Partition's information so they must be stopped first.
val partitionsToDelete = mutable.Set.empty[TopicPartition]
partitionsToStop.foreach { stopPartition =>
val topicPartition = stopPartition.topicPartition
var topicId: Option[Uuid] = None
if (stopPartition.deleteLocalLog) {
getPartition(topicPartition) match {
case hostedPartition: HostedPartition.Online[Partition] =>
if (allPartitions.remove(topicPartition, hostedPartition)) {
maybeRemoveTopicMetrics(topicPartition.topic)
// Logs are not deleted here. They are deleted in a single batch later on.
// This is done to avoid having to checkpoint for every deletions.
hostedPartition.partition.delete()
topicId = hostedPartition.partition.topicId
}
case _ =>
}
partitionsToDelete += topicPartition
}
// If we were the leader, we may have some operations still waiting for completion.
// We force completion to prevent them from timing out.
completeDelayedOperationsWhenNotPartitionLeader(topicPartition, topicId)
// Clean up per-partition expiration metrics regardless of whether the local log
// is deleted. This covers both partition deletion and reassignment (leader -> follower).
DelayedProduce.removePartitionMetrics(topicPartition)
DelayedRemoteListOffsets.removePartitionMetrics(topicPartition)
}
// Third delete the logs and checkpoint.
val errorMap = new mutable.HashMap[TopicPartition, Throwable]()
val remotePartitionsToStop = partitionsToStop.filter {
sp => logManager.getLog(sp.topicPartition).toScala.exists(unifiedLog => unifiedLog.remoteLogEnabled())
}
if (partitionsToDelete.nonEmpty) {
// Delete the logs and checkpoint.
logManager.asyncDelete(partitionsToDelete.asJava, false, (tp, e) => errorMap.put(tp, e))
}
remoteLogManager.foreach { rlm =>
// exclude the partitions with offline/error state
val partitions = remotePartitionsToStop.filterNot(sp => errorMap.contains(sp.topicPartition)).toSet.asJava
if (!partitions.isEmpty) {
rlm.stopPartitions(partitions, (tp, e) => errorMap.put(tp, e))
}
}
errorMap
}
def topicIdPartition(topicPartition: TopicPartition): TopicIdPartition = {
val topicId = metadataCache.getTopicId(topicPartition.topic())
new TopicIdPartition(topicId, topicPartition)
}
def getPartition(topicPartition: TopicPartition): HostedPartition[Partition] = {
Option(allPartitions.get(topicPartition)).getOrElse(new HostedPartition.None[Partition])
}
def isAddingReplica(topicPartition: TopicPartition, replicaId: Int): Boolean = {
getPartition(topicPartition) match {
case online: HostedPartition.Online[Partition] => online.partition.isAddingReplica(replicaId)
case _ => false
}
}
// Visible for testing
def createPartition(topicPartition: TopicPartition): Partition = {
val partition = Partition(topicPartition, time, this)
addOnlinePartition(topicPartition, partition)
partition
}
// Visible for testing
private[server] def addOnlinePartition(topicPartition: TopicPartition, partition: Partition): Unit = {
allPartitions.put(topicPartition, new HostedPartition.Online(partition))
}
def onlinePartition(topicPartition: TopicPartition): Option[Partition] = {
getPartition(topicPartition) match {
case online: HostedPartition.Online[Partition] => Some(online.partition)
case _ => None
}
}
// An iterator over all non offline partitions. This is a weakly consistent iterator; a partition made offline after
// the iterator has been constructed could still be returned by this iterator.
private def onlinePartitionsIterator: Iterator[Partition] = {
allPartitions.values.asScala.iterator.flatMap {
case online: HostedPartition.Online[Partition] => Some(online.partition)
case _ => None
}
}
private def offlinePartitionCount: Int = {
allPartitions.values.asScala.iterator.count(_.getClass == classOf[HostedPartition.Offline[Partition]])
}
def getPartitionOrException(topicPartition: TopicPartition): Partition = {
getPartitionOrError(topicPartition) match {
case Left(Errors.KAFKA_STORAGE_ERROR) =>
throw new KafkaStorageException(s"Partition $topicPartition is in an offline log directory")
case Left(error) =>
throw error.exception(s"Error while fetching partition state for $topicPartition")
case Right(partition) => partition
}
}
def getPartitionOrException(topicIdPartition: TopicIdPartition): Partition = {
getPartitionOrError(topicIdPartition.topicPartition()) match {
case Left(Errors.KAFKA_STORAGE_ERROR) =>
throw new KafkaStorageException(s"Partition ${topicIdPartition.topicPartition()} is in an offline log directory")
case Left(error) =>
throw error.exception(s"Error while fetching partition state for ${topicIdPartition.topicPartition()}")
case Right(partition) =>
// Get topic id for an existing partition from disk if topicId is none get it from the metadata cache
val topicId = partition.topicId.getOrElse(metadataCache.getTopicId(topicIdPartition.topic()))
// If topic id is set to zero_uuid fall back to non topic id aware behaviour
val topicIdNotProvided = topicIdPartition.topicId() == Uuid.ZERO_UUID
if (topicIdNotProvided || topicId == topicIdPartition.topicId()) {
partition
} else {
throw new UnknownTopicIdException(s"Partition $topicIdPartition's topic id doesn't match the one on disk $topicId.'")
}
}
}
def getPartitionOrError(topicPartition: TopicPartition): Either[Errors, Partition] = {
getPartition(topicPartition) match {
case online: HostedPartition.Online[Partition] =>
Right(online.partition)
case _: HostedPartition.Offline[Partition] =>
Left(Errors.KAFKA_STORAGE_ERROR)
case _: HostedPartition.None[Partition] if metadataCache.contains(topicPartition) =>
// The topic exists, but this broker is no longer a replica of it, so we return NOT_LEADER_OR_FOLLOWER which
// forces clients to refresh metadata to find the new location. This can happen, for example,
// during a partition reassignment if a produce request from the client is sent to a broker after
// the local replica has been deleted.
Left(Errors.NOT_LEADER_OR_FOLLOWER)
case _: HostedPartition.None[Partition] =>
Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)
}
}
def localLogOrException(topicPartition: TopicPartition): UnifiedLog = {
getPartitionOrException(topicPartition).localLogOrException
}
def futureLocalLogOrException(topicPartition: TopicPartition): UnifiedLog = {
getPartitionOrException(topicPartition).futureLocalLogOrException
}
def futureLogExists(topicPartition: TopicPartition): Boolean = {
getPartitionOrException(topicPartition).futureLog.isDefined
}
def futureLogOrException(topicPartition: TopicPartition): UnifiedLog = {
getPartitionOrException(topicPartition).futureLocalLogOrException
}
def localLog(topicPartition: TopicPartition): Option[UnifiedLog] = {
onlinePartition(topicPartition).flatMap(_.log)
}
def tryCompleteActions(): Unit = defaultActionQueue.tryCompleteActions()
def addToActionQueue(action: Runnable): Unit = defaultActionQueue.add(action)
/**
* Append messages to leader replicas of the partition, without waiting on replication.
*
* Noted that all pending delayed check operations are stored in a queue. All callers to ReplicaManager.appendRecordsToLeader()
* are expected to call ActionQueue.tryCompleteActions for all affected partitions, without holding any conflicting
* locks.
*
* @param requiredAcks the required acks -- it is only used to ensure that the append meets the
* required acks.
* @param internalTopicsAllowed boolean indicating whether internal topics can be appended to
* @param origin source of the append request (ie, client, replication, coordinator)
* @param entriesPerPartition the records per topic partition to be appended.
* If topic partition contains Uuid.ZERO_UUID as topicId the method
* will fall back to the old behaviour and rely on topic name.
* @param requestLocal container for the stateful instances scoped to this request -- this must correspond to the
* thread calling this method
* @param actionQueue the action queue to use. ReplicaManager#defaultActionQueue is used by default.
* @param verificationGuards the mapping from topic partition to verification guards if transaction verification is used
* @param transactionVersion the transaction version for the records (1 for TV1, 2 for TV2, etc.).
* Defaults to TV_UNKNOWN (-1) to force explicit specification.
* Used for epoch validation of transaction markers (KIP-1228).
*/
def appendRecordsToLeader(
requiredAcks: Short,
internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
requestLocal: RequestLocal = RequestLocal.noCaching,
actionQueue: ActionQueue = this.defaultActionQueue,
verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty,
transactionVersion: Short = TransactionVersion.TV_UNKNOWN
): Map[TopicIdPartition, LogAppendResult] = {
val startTimeMs = time.milliseconds
val localProduceResultsWithTopicId = appendToLocalLog(
internalTopicsAllowed = internalTopicsAllowed,
origin,
entriesPerPartition,
requiredAcks,
requestLocal,
verificationGuards.toMap,
transactionVersion
)
debug("Produce to local log in %d ms".format(time.milliseconds - startTimeMs))
addCompletePurgatoryAction(actionQueue, localProduceResultsWithTopicId)
localProduceResultsWithTopicId
}
/**
* Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
* the callback function will be triggered either when timeout or the required acks are satisfied;
* if the callback function itself is already synchronized on some object then pass this object to avoid deadlock.
*
* Noted that all pending delayed check operations are stored in a queue. All callers to ReplicaManager.appendRecords()
* are expected to call ActionQueue.tryCompleteActions for all affected partitions, without holding any conflicting
* locks.
*
* @param timeout maximum time we will wait to append before returning
* @param requiredAcks number of replicas who must acknowledge the append before sending the response
* @param internalTopicsAllowed boolean indicating whether internal topics can be appended to
* @param origin source of the append request (ie, client, replication, coordinator)
* @param entriesPerPartition the records per topic partition to be appended.
* If topic partition contains Uuid.ZERO_UUID as topicId the method
* will fall back to the old behaviour and rely on topic name.
* @param responseCallback callback for sending the response
* @param recordValidationStatsCallback callback for updating stats on record conversions
* @param requestLocal container for the stateful instances scoped to this request -- this must correspond to the
* thread calling this method
* @param verificationGuards the mapping from topic partition to verification guards if transaction verification is used
* @param transactionVersion the transaction version for the records (1 = TV1, 2 = TV2).
* Defaults to TV_UNKNOWN (-1) to force explicit specification.
* Used for epoch validation of transaction markers (KIP-1228).
*/
def appendRecords(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit,
recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (),
requestLocal: RequestLocal = RequestLocal.noCaching,
verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty,
transactionVersion: Short = TransactionVersion.TV_UNKNOWN): Unit = {
if (!isValidRequiredAcks(requiredAcks)) {
sendInvalidRequiredAcksResponse(entriesPerPartition, responseCallback)
return
}
val localProduceResults = appendRecordsToLeader(
requiredAcks,
internalTopicsAllowed,
origin,
entriesPerPartition,
requestLocal,
defaultActionQueue,
verificationGuards,
transactionVersion
)
val produceStatus = buildProducePartitionStatus(localProduceResults)
recordValidationStatsCallback(localProduceResults.map { case (k, v) =>
k -> v.logAppendSummary().recordValidationStats()
})
maybeAddDelayedProduce(
requiredAcks,
timeout,
entriesPerPartition,
localProduceResults,
produceStatus,
responseCallback
)
}
/**
* Handles the produce request by starting any transactional verification before appending.
*
* @param timeout maximum time we will wait to append before returning
* @param requiredAcks number of replicas who must acknowledge the append before sending the response
* @param internalTopicsAllowed boolean indicating whether internal topics can be appended to
* @param transactionalId the transactional ID for the produce request or null if there is none.
* @param entriesPerPartition the records per partition to be appended
* @param responseCallback callback for sending the response
* @param recordValidationStatsCallback callback for updating stats on record conversions
* @param requestLocal container for the stateful instances scoped to this request -- this must correspond to the
* thread calling this method
* @param transactionSupportedOperation determines the supported Operation based on the client's Request api version
*
* The responseCallback is wrapped so that it is scheduled on a request handler thread. There, it should be called with
* that request handler thread's thread local and not the one supplied to this method.
*/
def handleProduceAppend(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
transactionalId: String,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit,
recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (),
requestLocal: RequestLocal = RequestLocal.noCaching,
transactionSupportedOperation: TransactionSupportedOperation): Unit = {
val transactionalProducerInfo = mutable.HashSet[(Long, Short)]()
val topicPartitionBatchInfo = mutable.Map[TopicPartition, Int]()
val topicIds = entriesPerPartition.keys.map(tp => tp.topic() -> tp.topicId()).toMap
entriesPerPartition.foreachEntry { (topicIdPartition, records) =>
// Produce requests (only requests that require verification) should only have one batch per partition in "batches" but check all just to be safe.
val transactionalBatches = records.batches.asScala.filter(batch => batch.hasProducerId && batch.isTransactional)
transactionalBatches.foreach(batch => transactionalProducerInfo.add(batch.producerId, batch.producerEpoch))
if (transactionalBatches.nonEmpty) topicPartitionBatchInfo.put(topicIdPartition.topicPartition(), records.firstBatch.baseSequence)
}
if (transactionalProducerInfo.size > 1) {
throw new InvalidPidMappingException("Transactional records contained more than one producer ID")
}
def postVerificationCallback(newRequestLocal: RequestLocal,
results: (Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])): Unit = {
val (preAppendErrors, verificationGuards) = results
val errorResults: Map[TopicIdPartition, LogAppendResult] = preAppendErrors.map {
case (topicPartition, error) =>
// translate transaction coordinator errors to known producer response errors
val customException =
error match {
case Errors.INVALID_TXN_STATE => Some(error.exception("Partition was not added to the transaction"))
// Transaction verification can fail with a retriable error that older clients may not
// retry correctly. Translate these to an error which will cause such clients to retry
// the produce request. We pick `NOT_ENOUGH_REPLICAS` because it does not trigger a
// metadata refresh.
case Errors.NETWORK_EXCEPTION |
Errors.COORDINATOR_LOAD_IN_PROGRESS |
Errors.COORDINATOR_NOT_AVAILABLE |
Errors.NOT_COORDINATOR => Some(new NotEnoughReplicasException(
s"Unable to verify the partition has been added to the transaction. Underlying error: ${error.toString}"))
case Errors.CONCURRENT_TRANSACTIONS =>
if (!transactionSupportedOperation.supportsEpochBump) {
Some(new NotEnoughReplicasException(
s"Unable to verify the partition has been added to the transaction. Underlying error: ${error.toString}"))
} else {
// Don't convert the Concurrent Transaction exception for TV2. Because the error is very common during
// the transaction commit phase. Returning Concurrent Transaction is less confusing to the client.
None
}
case _ => None
}
new TopicIdPartition(topicIds.getOrElse(topicPartition.topic(), Uuid.ZERO_UUID), topicPartition) -> new LogAppendResult(
LogAppendSummary.fromAppendInfo(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO),
Optional.ofNullable(customException.getOrElse(error.exception)),
customException.isDefined
)
}
// In non-transaction paths, errorResults is typically empty, so we can
// directly use entriesPerPartition instead of creating a new filtered collection
val entriesWithoutErrorsPerPartition =
if (errorResults.nonEmpty) entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) }
else entriesPerPartition
val preAppendPartitionResponses = buildProducePartitionStatus(errorResults).map { case (k, status) => k -> status.responseStatus }
def newResponseCallback(responses: util.Map[TopicIdPartition, PartitionResponse]): Unit = {
responseCallback(preAppendPartitionResponses ++ responses.asScala)
}
appendRecords(
timeout = timeout,
requiredAcks = requiredAcks,
internalTopicsAllowed = internalTopicsAllowed,
origin = AppendOrigin.CLIENT,
entriesPerPartition = entriesWithoutErrorsPerPartition,
responseCallback = newResponseCallback,
recordValidationStatsCallback = recordValidationStatsCallback,
requestLocal = newRequestLocal,
verificationGuards = verificationGuards
)
}
if (transactionalProducerInfo.size < 1) {
postVerificationCallback(
requestLocal,
(Map.empty[TopicPartition, Errors], Map.empty[TopicPartition, VerificationGuard])
)
return
}
// Wrap the callback to be handled on an arbitrary request handler thread
// when transaction verification is complete. The request local passed in
// is only used when the callback is executed immediately.
val wrappedPostVerificationCallback = KafkaRequestHandler.wrapAsyncCallback(
postVerificationCallback,
requestLocal
)
val retryTimeoutMs = Math.min(addPartitionsToTxnConfig.addPartitionsToTxnRetryBackoffMaxMs(), config.requestTimeoutMs)
val addPartitionsRetryBackoffMs = addPartitionsToTxnConfig.addPartitionsToTxnRetryBackoffMs()
val startVerificationTimeMs = time.milliseconds
def maybeRetryOnConcurrentTransactions(results: (Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])): Unit = {
if (time.milliseconds() - startVerificationTimeMs >= retryTimeoutMs) {
// We've exceeded the retry timeout, so just call the callback with whatever results we have
wrappedPostVerificationCallback(results)
} else if (results._1.values.exists(_ == Errors.CONCURRENT_TRANSACTIONS)) {
// Retry the verification with backoff
scheduler.scheduleOnce("retry-add-partitions-to-txn", () => {
maybeSendPartitionsToTransactionCoordinator(
topicPartitionBatchInfo,
transactionalId,
transactionalProducerInfo.head._1,
transactionalProducerInfo.head._2,
maybeRetryOnConcurrentTransactions,
transactionSupportedOperation
)
}, addPartitionsRetryBackoffMs * 1L)
} else {
// We don't have concurrent transaction errors, so just call the callback with the results
wrappedPostVerificationCallback(results)
}
}
maybeSendPartitionsToTransactionCoordinator(
topicPartitionBatchInfo,
transactionalId,
transactionalProducerInfo.head._1,
transactionalProducerInfo.head._2,
// If we add partition directly from produce request,
// we should retry on concurrent transaction error here because:
// - the produce backoff adds too much delay
// - the produce request is expensive to retry
if (transactionSupportedOperation.supportsEpochBump) maybeRetryOnConcurrentTransactions else wrappedPostVerificationCallback,
transactionSupportedOperation
)
}
private def buildProducePartitionStatus(
results: Map[TopicIdPartition, LogAppendResult]
): Map[TopicIdPartition, ProducePartitionStatus] = {
results.map { case (topicIdPartition, result) =>
topicIdPartition -> new ProducePartitionStatus(
result.logAppendSummary.lastOffset + 1, // required offset
new PartitionResponse(
result.error,
result.logAppendSummary.firstOffset,
result.logAppendSummary.logAppendTime,
result.logAppendSummary.logStartOffset,
result.logAppendSummary.recordErrors,
result.errorMessage
)
)
}
}
private def addCompletePurgatoryAction(
actionQueue: ActionQueue,
appendResults: Map[TopicIdPartition, LogAppendResult]
): Unit = {
actionQueue.add {
() => appendResults.foreach { case (topicIdPartition, result) =>
val requestKey = new TopicPartitionOperationKey(topicIdPartition.topicPartition)
result.logAppendSummary.leaderHwChange match {
case LeaderHwChange.INCREASED =>
// some delayed operations may be unblocked after HW changed
delayedProducePurgatory.checkAndComplete(requestKey)
delayedFetchPurgatory.checkAndComplete(requestKey)
delayedDeleteRecordsPurgatory.checkAndComplete(requestKey)
if (topicIdPartition.topicId != Uuid.ZERO_UUID) delayedShareFetchPurgatory.checkAndComplete(new DelayedShareFetchPartitionKey(
topicIdPartition.topicId, topicIdPartition.partition))
case LeaderHwChange.SAME =>
// probably unblock some follower fetch requests since log end offset has been updated
delayedFetchPurgatory.checkAndComplete(requestKey)
case LeaderHwChange.NONE =>
// nothing
}
}
}
}
private def maybeAddDelayedProduce(
requiredAcks: Short,
timeoutMs: Long,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
initialAppendResults: Map[TopicIdPartition, LogAppendResult],
initialProduceStatus: Map[TopicIdPartition, ProducePartitionStatus],
responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit,
): Unit = {
if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) {
// Create delayed produce operation
//
// This delegate is invoked by DelayedProduce to verify if the produce operation can be completed.
// Defined here to provide access to ReplicaManager#getPartitionOrError, which is otherwise inaccessible to the caller.
def delegate(tp: TopicPartition, requiredOffset: Long) : Result = {
val (hasEnough, error) = getPartitionOrError(tp).fold(
// Please refer to the documentation in `DelayedProduce#tryComplete` for a comprehensive description of these cases.
// Case A or Case B
err => (false, err),
// Case B or Case C
partition => partition.checkEnoughReplicasReachOffset(requiredOffset))
new Result(hasEnough, error)
}
val delayedProduce = new DelayedProduce(timeoutMs, initialProduceStatus.asJava, delegate, responseCallback.asJava)
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toList
// try to complete the request immediately, otherwise put it into the purgatory
// this is because while the delayed produce operation is being created, new
// requests may arrive and hence make this operation completable.
delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys.asJava)
} else {
// we can respond immediately
val produceResponseStatus = new util.HashMap[TopicIdPartition, PartitionResponse]
initialProduceStatus.foreach { case (k, status) => produceResponseStatus.put(k, status.responseStatus) }
responseCallback(produceResponseStatus)
}
}
private def sendInvalidRequiredAcksResponse(
entries: Map[TopicIdPartition, MemoryRecords],
responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit): Unit = {
// If required.acks is outside accepted range, something is wrong with the client
// Just return an error and don't handle the request at all
val responseStatus = new util.HashMap[TopicIdPartition, PartitionResponse]
entries.foreach { case(topicIdPartition, _) =>
responseStatus.put(topicIdPartition, new PartitionResponse(
Errors.INVALID_REQUIRED_ACKS,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.firstOffset,
RecordBatch.NO_TIMESTAMP,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.logStartOffset)
)
}
responseCallback(responseStatus)
}
/**
*
* @param topicPartition the topic partition to maybe verify or add
* @param transactionalId the transactional id for the transaction
* @param producerId the producer id for the producer writing to the transaction
* @param producerEpoch the epoch of the producer writing to the transaction
* @param baseSequence the base sequence of the first record in the batch we are trying to append
* @param callback the method to execute once the verification is either completed or returns an error
* @param transactionSupportedOperation determines the supported operation based on the client's Request API version
*
* If this is the first time a partition appears in a transaction, it must be verified or added to the partition depending on the
* transactionSupported operation.
* If verifying, when the verification returns, the callback will be supplied the error if it exists or Errors.NONE.
* If the verification guard exists, it will also be supplied. Otherwise the SENTINEL verification guard will be returned.
* This guard can not be used for verification and any appends that attempt to use it will fail.
*
* If adding, the callback will be supplied the error if it exists or Errors.NONE.
*/
def maybeSendPartitionToTransactionCoordinator(
topicPartition: TopicPartition,
transactionalId: String,
producerId: Long,
producerEpoch: Short,
baseSequence: Int,
callback: ((Errors, VerificationGuard)) => Unit,
transactionSupportedOperation: TransactionSupportedOperation
): Unit = {
def generalizedCallback(results: (Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])): Unit = {
val (preAppendErrors, verificationGuards) = results
callback((
preAppendErrors.getOrElse(topicPartition, Errors.NONE),
verificationGuards.getOrElse(topicPartition, VerificationGuard.SENTINEL)
))
}
maybeSendPartitionsToTransactionCoordinator(
Map(topicPartition -> baseSequence),
transactionalId,
producerId,
producerEpoch,
generalizedCallback,
transactionSupportedOperation
)
}
/**
*
* @param topicPartitionBatchInfo the topic partitions to maybe verify or add mapped to the base sequence of their first record batch
* @param transactionalId the transactional id for the transaction
* @param producerId the producer id for the producer writing to the transaction
* @param producerEpoch the epoch of the producer writing to the transaction
* @param callback the method to execute once the verification is either completed or returns an error
* @param transactionSupportedOperation determines the supported operation based on the client's Request API version
*
* If this is the first time the partitions appear in a transaction, they must be verified or added to the partition depending on the
* transactionSupported operation.
* If verifying, when the verification returns, the callback will be supplied the errors per topic partition if there were errors.
* The callback will also be supplied the verification guards per partition if they exist. It is possible to have an
* error and a verification guard for a topic partition if the topic partition was unable to be verified by the transaction
* coordinator. Transaction coordinator errors are mapped to append-friendly errors.
*
* If adding, the callback will be e supplied the errors per topic partition if there were errors.
*/
private def maybeSendPartitionsToTransactionCoordinator(