-
Notifications
You must be signed in to change notification settings - Fork 14.1k
/
Copy pathReplicaManager.scala
2037 lines (1823 loc) · 101 KB
/
ReplicaManager.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.server
import java.io.File
import java.util.Optional
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.Lock
import com.yammer.metrics.core.Meter
import kafka.api._
import kafka.cluster.{BrokerEndPoint, Partition}
import kafka.common.RecordValidationException
import kafka.controller.{KafkaController, StateChangeLogger}
import kafka.log._
import kafka.metrics.KafkaMetricsGroup
import kafka.server.{FetchMetadata => SFetchMetadata}
import kafka.server.HostedPartition.Online
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile, OffsetCheckpoints}
import kafka.server.metadata.ConfigRepository
import kafka.utils._
import kafka.utils.Implicits._
import kafka.zk.KafkaZkClient
import org.apache.kafka.common.{ElectionType, IsolationLevel, Node, TopicPartition, Uuid}
import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult
import org.apache.kafka.common.message.{DescribeLogDirsResponseData, DescribeProducersResponseData, FetchResponseData, LeaderAndIsrResponseData}
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError
import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult}
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState
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.FileRecords.TimestampAndOffset
import org.apache.kafka.common.record._
import org.apache.kafka.common.replica.PartitionView.DefaultPartitionView
import org.apache.kafka.common.replica.ReplicaView.DefaultReplicaView
import org.apache.kafka.common.replica.{ClientMetadata, _}
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.Time
import scala.jdk.CollectionConverters._
import scala.collection.{Map, Seq, Set, mutable}
import scala.compat.java8.OptionConverters._
/*
* Result metadata of a log append operation on the log
*/
case class LogAppendResult(info: LogAppendInfo, exception: Option[Throwable] = None) {
def error: Errors = exception match {
case None => Errors.NONE
case Some(e) => Errors.forException(e)
}
}
case class LogDeleteRecordsResult(requestedOffset: Long, lowWatermark: Long, exception: Option[Throwable] = None) {
def error: Errors = exception match {
case None => Errors.NONE
case Some(e) => Errors.forException(e)
}
}
/**
* Result metadata of a log read operation on the log
* @param info @FetchDataInfo returned by the @Log read
* @param divergingEpoch Optional epoch and end offset which indicates the largest epoch such
* that subsequent records are known to diverge on the follower/consumer
* @param highWatermark high watermark of the local replica
* @param leaderLogStartOffset The log start offset of the leader at the time of the read
* @param leaderLogEndOffset The log end offset of the leader at the time of the read
* @param followerLogStartOffset The log start offset of the follower taken from the Fetch request
* @param fetchTimeMs The time the fetch was received
* @param lastStableOffset Current LSO or None if the result has an exception
* @param preferredReadReplica the preferred read replica to be used for future fetches
* @param exception Exception if error encountered while reading from the log
*/
case class LogReadResult(info: FetchDataInfo,
divergingEpoch: Option[FetchResponseData.EpochEndOffset],
highWatermark: Long,
leaderLogStartOffset: Long,
leaderLogEndOffset: Long,
followerLogStartOffset: Long,
fetchTimeMs: Long,
lastStableOffset: Option[Long],
preferredReadReplica: Option[Int] = None,
exception: Option[Throwable] = None) {
def error: Errors = exception match {
case None => Errors.NONE
case Some(e) => Errors.forException(e)
}
def toFetchPartitionData(isReassignmentFetch: Boolean): FetchPartitionData = FetchPartitionData(
this.error,
this.highWatermark,
this.leaderLogStartOffset,
this.info.records,
this.divergingEpoch,
this.lastStableOffset,
this.info.abortedTransactions,
this.preferredReadReplica,
isReassignmentFetch)
def withEmptyFetchInfo: LogReadResult =
copy(info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY))
override def toString = {
"LogReadResult(" +
s"info=$info, " +
s"divergingEpoch=$divergingEpoch, " +
s"highWatermark=$highWatermark, " +
s"leaderLogStartOffset=$leaderLogStartOffset, " +
s"leaderLogEndOffset=$leaderLogEndOffset, " +
s"followerLogStartOffset=$followerLogStartOffset, " +
s"fetchTimeMs=$fetchTimeMs, " +
s"preferredReadReplica=$preferredReadReplica, " +
s"lastStableOffset=$lastStableOffset, " +
s"error=$error" +
")"
}
}
case class FetchPartitionData(error: Errors = Errors.NONE,
highWatermark: Long,
logStartOffset: Long,
records: Records,
divergingEpoch: Option[FetchResponseData.EpochEndOffset],
lastStableOffset: Option[Long],
abortedTransactions: Option[List[FetchResponseData.AbortedTransaction]],
preferredReadReplica: Option[Int],
isReassignmentFetch: Boolean)
/**
* Trait to represent the state of hosted partitions. We create a concrete (active) Partition
* instance when the broker receives a LeaderAndIsr request from the controller or a metadata
* log record from the Quorum controller indicating that the broker should be either a leader
* or follower of a partition.
*/
sealed trait HostedPartition
/**
* Trait to represent a partition that isn't Offline -- i.e. it is either Online or it is Deferred.
*/
sealed trait NonOffline extends HostedPartition {
val partition: Partition
}
object HostedPartition {
/**
* This broker does not have any state for this partition locally.
*/
final object None extends HostedPartition
/**
* This broker hosts the partition and it is online.
*/
final case class Online(partition: Partition) extends NonOffline
/**
* This broker hosted the partition (or will soon host it if it is new) but
* it is deferring metadata changes until it catches up on the Raft-based metadata
* log. This state only applies to brokers that are using a Raft-based metadata
* quorum; it never happens when using ZooKeeper. The isNew value indicates
* if the partition needs to be created when we apply the deferred changes.
*/
final case class Deferred(partition: Partition, isNew: Boolean) extends NonOffline
/**
* This broker hosts the partition, but it is in an offline log directory.
*/
final object Offline extends HostedPartition
}
object ReplicaManager {
val HighWatermarkFilename = "replication-offset-checkpoint"
}
class ReplicaManager(val config: KafkaConfig,
metrics: Metrics,
time: Time,
val zkClient: Option[KafkaZkClient],
scheduler: Scheduler,
val logManager: LogManager,
val isShuttingDown: AtomicBoolean,
quotaManagers: QuotaManagers,
val brokerTopicStats: BrokerTopicStats,
val metadataCache: MetadataCache,
logDirFailureChannel: LogDirFailureChannel,
val delayedProducePurgatory: DelayedOperationPurgatory[DelayedProduce],
val delayedFetchPurgatory: DelayedOperationPurgatory[DelayedFetch],
val delayedDeleteRecordsPurgatory: DelayedOperationPurgatory[DelayedDeleteRecords],
val delayedElectLeaderPurgatory: DelayedOperationPurgatory[DelayedElectLeader],
threadNamePrefix: Option[String],
configRepository: ConfigRepository,
val alterIsrManager: AlterIsrManager) extends Logging with KafkaMetricsGroup {
def this(config: KafkaConfig,
metrics: Metrics,
time: Time,
zkClient: Option[KafkaZkClient],
scheduler: Scheduler,
logManager: LogManager,
isShuttingDown: AtomicBoolean,
quotaManagers: QuotaManagers,
brokerTopicStats: BrokerTopicStats,
metadataCache: MetadataCache,
logDirFailureChannel: LogDirFailureChannel,
alterIsrManager: AlterIsrManager,
configRepository: ConfigRepository,
threadNamePrefix: Option[String] = None) = {
this(config, metrics, time, zkClient, scheduler, logManager, isShuttingDown,
quotaManagers, brokerTopicStats, metadataCache, logDirFailureChannel,
DelayedOperationPurgatory[DelayedProduce](
purgatoryName = "Produce", brokerId = config.brokerId,
purgeInterval = config.producerPurgatoryPurgeIntervalRequests),
DelayedOperationPurgatory[DelayedFetch](
purgatoryName = "Fetch", brokerId = config.brokerId,
purgeInterval = config.fetchPurgatoryPurgeIntervalRequests),
DelayedOperationPurgatory[DelayedDeleteRecords](
purgatoryName = "DeleteRecords", brokerId = config.brokerId,
purgeInterval = config.deleteRecordsPurgatoryPurgeIntervalRequests),
DelayedOperationPurgatory[DelayedElectLeader](
purgatoryName = "ElectLeader", brokerId = config.brokerId),
threadNamePrefix, configRepository, alterIsrManager)
}
/* epoch of the controller that last changed the leader */
@volatile private[server] var controllerEpoch: Int = KafkaController.InitialControllerEpoch
protected val localBrokerId = config.brokerId
protected val allPartitions = new Pool[TopicPartition, HostedPartition](
valueFactory = Some(tp => HostedPartition.Online(Partition(tp, time, configRepository, this)))
)
protected val replicaStateChangeLock = new Object
val replicaFetcherManager = createReplicaFetcherManager(metrics, time, threadNamePrefix, 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.map(dir =>
(dir.getAbsolutePath, new OffsetCheckpointFile(new File(dir, ReplicaManager.HighWatermarkFilename), logDirFailureChannel))).toMap
this.logIdent = s"[ReplicaManager broker=$localBrokerId] "
protected val stateChangeLogger = new StateChangeLogger(localBrokerId, inControllerContext = false, None)
private var logDirFailureHandler: LogDirFailureHandler = null
private class LogDirFailureHandler(name: String, haltBrokerOnDirFailure: Boolean) extends ShutdownableThread(name) {
override def doWork(): Unit = {
val newOfflineLogDir = logDirFailureChannel.takeNextOfflineLogDir()
if (haltBrokerOnDirFailure) {
fatal(s"Halting broker because dir $newOfflineLogDir is offline")
Exit.halt(1)
}
handleLogDirFailure(newOfflineLogDir)
}
}
// Visible for testing
private[server] val replicaSelectorOpt: Option[ReplicaSelector] = createReplicaSelector()
newGauge("LeaderCount", () => leaderPartitionsIterator.size)
// Visible for testing
private[kafka] val partitionCount = newGauge("PartitionCount", () => allPartitions.size)
newGauge("OfflineReplicaCount", () => offlinePartitionCount)
newGauge("UnderReplicatedPartitions", () => underReplicatedPartitionCount)
newGauge("UnderMinIsrPartitionCount", () => leaderPartitionsIterator.count(_.isUnderMinIsr))
newGauge("AtMinIsrPartitionCount", () => leaderPartitionsIterator.count(_.isAtMinIsr))
newGauge("ReassigningPartitions", () => reassigningPartitionsCount)
def reassigningPartitionsCount: Int = leaderPartitionsIterator.count(_.isReassigning)
val isrExpandRate: Meter = newMeter("IsrExpandsPerSec", "expands", TimeUnit.SECONDS)
val isrShrinkRate: Meter = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS)
val failedIsrUpdatesRate: Meter = newMeter("FailedIsrUpdatesPerSec", "failedUpdates", TimeUnit.SECONDS)
def underReplicatedPartitionCount: Int = leaderPartitionsIterator.count(_.isUnderReplicated)
def startHighWatermarkCheckPointThread(): Unit = {
if (highWatermarkCheckPointThreadStarted.compareAndSet(false, true))
scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks _, period = config.replicaHighWatermarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS)
}
// 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
def shutdownIdleReplicaAlterLogDirsThread(): Unit = {
replicaAlterLogDirsManager.shutdownIdleFetcherThreads()
}
def getLog(topicPartition: TopicPartition): Option[Log] = logManager.getLog(topicPartition)
def hasDelayedElectionOperations: Boolean = delayedElectLeaderPurgatory.numDelayed != 0
def tryCompleteElection(key: DelayedOperationKey): Unit = {
val completed = delayedElectLeaderPurgatory.checkAndComplete(key)
debug("Request key %s unblocked %d ElectLeader.".format(key.keyLabel, completed))
}
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 _, period = config.replicaLagTimeMaxMs / 2, unit = TimeUnit.MILLISECONDS)
scheduler.schedule("shutdown-idle-replica-alter-log-dirs-thread", shutdownIdleReplicaAlterLogDirsThread _, period = 10000L, unit = TimeUnit.MILLISECONDS)
// If inter-broker protocol (IBP) < 1.0, the controller will send LeaderAndIsrRequest V0 which does not include isNew field.
// In this case, the broker receiving the request cannot determine whether it is safe to create a partition if a log directory has failed.
// Thus, we choose to halt the broker on any log diretory failure if IBP < 1.0
val haltBrokerOnFailure = config.interBrokerProtocolVersion < KAFKA_1_0_IV0
logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", haltBrokerOnFailure)
logDirFailureHandler.start()
}
private def maybeRemoveTopicMetrics(topic: String): Unit = {
val topicHasNonOfflinePartition = allPartitions.values.exists {
case nonOffline: NonOffline => topic == nonOffline.partition.topic
case HostedPartition.None | HostedPartition.Offline => false
}
if (!topicHasNonOfflinePartition) // nothing online or deferred
brokerTopicStats.removeMetrics(topic)
}
protected def completeDelayedFetchOrProduceRequests(topicPartition: TopicPartition): Unit = {
val topicPartitionOperationKey = TopicPartitionOperationKey(topicPartition)
delayedProducePurgatory.checkAndComplete(topicPartitionOperationKey)
delayedFetchPurgatory.checkAndComplete(topicPartitionOperationKey)
}
def stopReplicas(correlationId: Int,
controllerId: Int,
controllerEpoch: Int,
brokerEpoch: Long,
partitionStates: Map[TopicPartition, StopReplicaPartitionState]
): (mutable.Map[TopicPartition, Errors], Errors) = {
replicaStateChangeLock synchronized {
stateChangeLogger.info(s"Handling StopReplica request correlationId $correlationId from controller " +
s"$controllerId for ${partitionStates.size} partitions")
if (stateChangeLogger.isTraceEnabled)
partitionStates.forKeyValue { (topicPartition, partitionState) =>
stateChangeLogger.trace(s"Received StopReplica request $partitionState " +
s"correlation id $correlationId from controller $controllerId " +
s"epoch $controllerEpoch for partition $topicPartition")
}
val responseMap = new collection.mutable.HashMap[TopicPartition, Errors]
if (controllerEpoch < this.controllerEpoch) {
stateChangeLogger.warn(s"Ignoring StopReplica request from " +
s"controller $controllerId with correlation id $correlationId " +
s"since its controller epoch $controllerEpoch is old. " +
s"Latest known controller epoch is ${this.controllerEpoch}")
(responseMap, Errors.STALE_CONTROLLER_EPOCH)
} else {
this.controllerEpoch = controllerEpoch
val stoppedPartitions = mutable.Map.empty[TopicPartition, Boolean]
partitionStates.forKeyValue { (topicPartition, partitionState) =>
val deletePartition = partitionState.deletePartition()
getPartition(topicPartition) match {
case HostedPartition.Offline =>
stateChangeLogger.warn(s"Ignoring StopReplica request (delete=$deletePartition) from " +
s"controller $controllerId with correlation id $correlationId " +
s"epoch $controllerEpoch for partition $topicPartition as the local replica for the " +
"partition is in an offline log directory")
responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR)
case HostedPartition.Online(partition) =>
val currentLeaderEpoch = partition.getLeaderEpoch
val requestLeaderEpoch = partitionState.leaderEpoch
// When a topic is deleted, the leader epoch is not incremented. To circumvent this,
// a sentinel value (EpochDuringDelete) overwriting any previous epoch is used.
// When an older version of the StopReplica request which does not contain the leader
// epoch, a sentinel value (NoEpoch) is used and bypass the epoch validation.
if (requestLeaderEpoch == LeaderAndIsr.EpochDuringDelete ||
requestLeaderEpoch == LeaderAndIsr.NoEpoch ||
requestLeaderEpoch > currentLeaderEpoch) {
stoppedPartitions += topicPartition -> deletePartition
// Assume that everything will go right. It is overwritten in case of an error.
responseMap.put(topicPartition, Errors.NONE)
} else if (requestLeaderEpoch < currentLeaderEpoch) {
stateChangeLogger.warn(s"Ignoring StopReplica request (delete=$deletePartition) from " +
s"controller $controllerId with correlation id $correlationId " +
s"epoch $controllerEpoch for partition $topicPartition since its associated " +
s"leader epoch $requestLeaderEpoch is smaller than the current " +
s"leader epoch $currentLeaderEpoch")
responseMap.put(topicPartition, Errors.FENCED_LEADER_EPOCH)
} else {
stateChangeLogger.info(s"Ignoring StopReplica request (delete=$deletePartition) from " +
s"controller $controllerId with correlation id $correlationId " +
s"epoch $controllerEpoch for partition $topicPartition since its associated " +
s"leader epoch $requestLeaderEpoch matches the current leader epoch")
responseMap.put(topicPartition, Errors.FENCED_LEADER_EPOCH)
}
case _: HostedPartition.Deferred =>
throw new IllegalStateException("We should never be deferring partition metadata changes and stopping a replica when using ZooKeeper")
case HostedPartition.None =>
// Delete log and corresponding folders in case replica manager doesn't hold them anymore.
// This could happen when topic is being deleted while broker is down and recovers.
stoppedPartitions += topicPartition -> deletePartition
responseMap.put(topicPartition, Errors.NONE)
}
}
stopPartitions(stoppedPartitions).foreach { case (topicPartition, e) =>
if (e.isInstanceOf[KafkaStorageException]) {
stateChangeLogger.error(s"Ignoring StopReplica request (delete=true) from " +
s"controller $controllerId with correlation id $correlationId " +
s"epoch $controllerEpoch for partition $topicPartition as the local replica for the " +
"partition is in an offline log directory")
} else {
stateChangeLogger.error(s"Ignoring StopReplica request (delete=true) from " +
s"controller $controllerId with correlation id $correlationId " +
s"epoch $controllerEpoch for partition $topicPartition due to an unexpected " +
s"${e.getClass.getName} exception: ${e.getMessage}")
responseMap.put(topicPartition, Errors.forException(e))
}
responseMap.put(topicPartition, Errors.forException(e))
}
(responseMap, Errors.NONE)
}
}
}
/**
* Stop the given partitions.
*
* @param partitionsToStop A map from a topic partition to a boolean indicating
* whether the partition should be deleted.
*
* @return A map from partitions to exceptions which occurred.
* If no errors occurred, the map will be empty.
*/
protected def stopPartitions(partitionsToStop: Map[TopicPartition, Boolean]): Map[TopicPartition, Throwable] = {
// First stop fetchers for all partitions.
val partitions = partitionsToStop.keySet
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.forKeyValue { (topicPartition, shouldDelete) =>
if (shouldDelete) {
getPartition(topicPartition) match {
case hostedPartition: NonOffline =>
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()
}
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.
completeDelayedFetchOrProduceRequests(topicPartition)
}
// Third delete the logs and checkpoint.
val errorMap = new mutable.HashMap[TopicPartition, Throwable]()
if (partitionsToDelete.nonEmpty) {
// Delete the logs and checkpoint.
logManager.asyncDelete(partitionsToDelete, (tp, e) => errorMap.put(tp, e))
}
errorMap
}
def getPartition(topicPartition: TopicPartition): HostedPartition = {
Option(allPartitions.get(topicPartition)).getOrElse(HostedPartition.None)
}
def isAddingReplica(topicPartition: TopicPartition, replicaId: Int): Boolean = {
getPartition(topicPartition) match {
case Online(partition) => partition.isAddingReplica(replicaId)
case _ => false
}
}
// Visible for testing
def createPartition(topicPartition: TopicPartition): Partition = {
val partition = Partition(topicPartition, time, configRepository, this)
allPartitions.put(topicPartition, HostedPartition.Online(partition))
partition
}
def onlinePartition(topicPartition: TopicPartition): Option[Partition] = {
getPartition(topicPartition) match {
case HostedPartition.Online(partition) => Some(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.iterator.flatMap {
case HostedPartition.Online(partition) => Some(partition)
case _ => None
}
}
private def offlinePartitionCount: Int = {
allPartitions.values.iterator.count(_ == HostedPartition.Offline)
}
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 getPartitionOrError(topicPartition: TopicPartition): Either[Errors, Partition] = {
getPartition(topicPartition) match {
case HostedPartition.Online(partition) =>
Right(partition)
case HostedPartition.Offline =>
Left(Errors.KAFKA_STORAGE_ERROR)
case HostedPartition.None 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.Deferred =>
// The topic exists, but this broker is deferring metadata changes for it, so we return NOT_LEADER_OR_FOLLOWER
// which forces clients to refresh metadata.
Left(Errors.NOT_LEADER_OR_FOLLOWER)
case HostedPartition.None =>
Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)
}
}
def localLogOrException(topicPartition: TopicPartition): Log = {
getPartitionOrException(topicPartition).localLogOrException
}
def futureLocalLogOrException(topicPartition: TopicPartition): Log = {
getPartitionOrException(topicPartition).futureLocalLogOrException
}
def futureLogExists(topicPartition: TopicPartition): Boolean = {
getPartitionOrException(topicPartition).futureLog.isDefined
}
def localLog(topicPartition: TopicPartition): Option[Log] = {
onlinePartition(topicPartition).flatMap(_.log)
}
def getLogDir(topicPartition: TopicPartition): Option[String] = {
localLog(topicPartition).map(_.parentDir)
}
/**
* TODO: move this action queue to handle thread so we can simplify concurrency handling
*/
private val actionQueue = new ActionQueue
def tryCompleteActions(): Unit = actionQueue.tryCompleteActions()
/**
* 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.
*/
def appendRecords(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
responseCallback: Map[TopicPartition, PartitionResponse] => Unit,
delayedProduceLock: Option[Lock] = None,
recordConversionStatsCallback: Map[TopicPartition, RecordConversionStats] => Unit = _ => ()): Unit = {
if (isValidRequiredAcks(requiredAcks)) {
val sTime = time.milliseconds
val localProduceResults = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed,
origin, entriesPerPartition, requiredAcks)
debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
val produceStatus = localProduceResults.map { case (topicPartition, result) =>
topicPartition -> ProducePartitionStatus(
result.info.lastOffset + 1, // required offset
new PartitionResponse(
result.error,
result.info.firstOffset.map(_.messageOffset).getOrElse(-1),
result.info.logAppendTime,
result.info.logStartOffset,
result.info.recordErrors.asJava,
result.info.errorMessage
)
) // response status
}
actionQueue.add {
() =>
localProduceResults.foreach {
case (topicPartition, result) =>
val requestKey = TopicPartitionOperationKey(topicPartition)
result.info.leaderHwChange match {
case LeaderHwChange.Increased =>
// some delayed operations may be unblocked after HW changed
delayedProducePurgatory.checkAndComplete(requestKey)
delayedFetchPurgatory.checkAndComplete(requestKey)
delayedDeleteRecordsPurgatory.checkAndComplete(requestKey)
case LeaderHwChange.Same =>
// probably unblock some follower fetch requests since log end offset has been updated
delayedFetchPurgatory.checkAndComplete(requestKey)
case LeaderHwChange.None =>
// nothing
}
}
}
recordConversionStatsCallback(localProduceResults.map { case (k, v) => k -> v.info.recordConversionStats })
if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
// create delayed produce operation
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback, delayedProduceLock)
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
val producerRequestKeys = entriesPerPartition.keys.map(TopicPartitionOperationKey(_)).toSeq
// 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)
} else {
// we can respond immediately
val produceResponseStatus = produceStatus.map { case (k, status) => k -> status.responseStatus }
responseCallback(produceResponseStatus)
}
} else {
// 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 = entriesPerPartition.map { case (topicPartition, _) =>
topicPartition -> new PartitionResponse(
Errors.INVALID_REQUIRED_ACKS,
LogAppendInfo.UnknownLogAppendInfo.firstOffset.map(_.messageOffset).getOrElse(-1),
RecordBatch.NO_TIMESTAMP,
LogAppendInfo.UnknownLogAppendInfo.logStartOffset
)
}
responseCallback(responseStatus)
}
}
/**
* Delete records on leader replicas of the partition, and wait for delete records operation be propagated to other replicas;
* the callback function will be triggered either when timeout or logStartOffset of all live replicas have reached the specified offset
*/
private def deleteRecordsOnLocalLog(offsetPerPartition: Map[TopicPartition, Long]): Map[TopicPartition, LogDeleteRecordsResult] = {
trace("Delete records on local logs to offsets [%s]".format(offsetPerPartition))
offsetPerPartition.map { case (topicPartition, requestedOffset) =>
// reject delete records operation on internal topics
if (Topic.isInternal(topicPartition.topic)) {
(topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(new InvalidTopicException(s"Cannot delete records of internal topic ${topicPartition.topic}"))))
} else {
try {
val partition = getPartitionOrException(topicPartition)
val logDeleteResult = partition.deleteRecordsOnLeader(requestedOffset)
(topicPartition, logDeleteResult)
} catch {
case e@ (_: UnknownTopicOrPartitionException |
_: NotLeaderOrFollowerException |
_: OffsetOutOfRangeException |
_: PolicyViolationException |
_: KafkaStorageException) =>
(topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(e)))
case t: Throwable =>
error("Error processing delete records operation on partition %s".format(topicPartition), t)
(topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(t)))
}
}
}
}
// If there exists a topic partition that meets the following requirement,
// we need to put a delayed DeleteRecordsRequest and wait for the delete records operation to complete
//
// 1. the delete records operation on this partition is successful
// 2. low watermark of this partition is smaller than the specified offset
private def delayedDeleteRecordsRequired(localDeleteRecordsResults: Map[TopicPartition, LogDeleteRecordsResult]): Boolean = {
localDeleteRecordsResults.exists{ case (_, deleteRecordsResult) =>
deleteRecordsResult.exception.isEmpty && deleteRecordsResult.lowWatermark < deleteRecordsResult.requestedOffset
}
}
/**
* For each pair of partition and log directory specified in the map, if the partition has already been created on
* this broker, move its log files to the specified log directory. Otherwise, record the pair in the memory so that
* the partition will be created in the specified log directory when broker receives LeaderAndIsrRequest for the partition later.
*/
def alterReplicaLogDirs(partitionDirs: Map[TopicPartition, String]): Map[TopicPartition, Errors] = {
replicaStateChangeLock synchronized {
partitionDirs.map { case (topicPartition, destinationDir) =>
try {
/* If the topic name is exceptionally long, we can't support altering the log directory.
* See KAFKA-4893 for details.
* TODO: fix this by implementing topic IDs. */
if (Log.logFutureDirName(topicPartition).size > 255)
throw new InvalidTopicException("The topic name is too long.")
if (!logManager.isLogDirOnline(destinationDir))
throw new KafkaStorageException(s"Log directory $destinationDir is offline")
getPartition(topicPartition) match {
case HostedPartition.Online(partition) =>
// Stop current replica movement if the destinationDir is different from the existing destination log directory
if (partition.futureReplicaDirChanged(destinationDir)) {
replicaAlterLogDirsManager.removeFetcherForPartitions(Set(topicPartition))
partition.removeFutureLocalReplica()
}
case HostedPartition.Offline =>
throw new KafkaStorageException(s"Partition $topicPartition is offline")
case _: HostedPartition.Deferred =>
throw new IllegalStateException(s"Partition $topicPartition is deferred")
case HostedPartition.None => // Do nothing
}
// If the log for this partition has not been created yet:
// 1) Record the destination log directory in the memory so that the partition will be created in this log directory
// when broker receives LeaderAndIsrRequest for this partition later.
// 2) Respond with NotLeaderOrFollowerException for this partition in the AlterReplicaLogDirsResponse
logManager.maybeUpdatePreferredLogDir(topicPartition, destinationDir)
// throw NotLeaderOrFollowerException if replica does not exist for the given partition
val partition = getPartitionOrException(topicPartition)
partition.localLogOrException
// If the destinationLDir is different from the current log directory of the replica:
// - If there is no offline log directory, create the future log in the destinationDir (if it does not exist) and
// start ReplicaAlterDirThread to move data of this partition from the current log to the future log
// - Otherwise, return KafkaStorageException. We do not create the future log while there is offline log directory
// so that we can avoid creating future log for the same partition in multiple log directories.
val highWatermarkCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints)
if (partition.maybeCreateFutureReplica(destinationDir, highWatermarkCheckpoints)) {
val futureLog = futureLocalLogOrException(topicPartition)
logManager.abortAndPauseCleaning(topicPartition)
val initialFetchState = InitialFetchState(BrokerEndPoint(config.brokerId, "localhost", -1),
partition.getLeaderEpoch, futureLog.highWatermark)
replicaAlterLogDirsManager.addFetcherForPartitions(Map(topicPartition -> initialFetchState))
}
(topicPartition, Errors.NONE)
} catch {
case e@(_: InvalidTopicException |
_: LogDirNotFoundException |
_: ReplicaNotAvailableException |
_: KafkaStorageException) =>
warn(s"Unable to alter log dirs for $topicPartition", e)
(topicPartition, Errors.forException(e))
case e: NotLeaderOrFollowerException =>
// Retaining REPLICA_NOT_AVAILABLE exception for ALTER_REPLICA_LOG_DIRS for compatibility
warn(s"Unable to alter log dirs for $topicPartition", e)
(topicPartition, Errors.REPLICA_NOT_AVAILABLE)
case t: Throwable =>
error("Error while changing replica dir for partition %s".format(topicPartition), t)
(topicPartition, Errors.forException(t))
}
}
}
}
/*
* Get the LogDirInfo for the specified list of partitions.
*
* Each LogDirInfo specifies the following information for a given log directory:
* 1) Error of the log directory, e.g. whether the log is online or offline
* 2) size and lag of current and future logs for each partition in the given log directory. Only logs of the queried partitions
* are included. There may be future logs (which will replace the current logs of the partition in the future) on the broker after KIP-113 is implemented.
*/
def describeLogDirs(partitions: Set[TopicPartition]): List[DescribeLogDirsResponseData.DescribeLogDirsResult] = {
val logsByDir = logManager.allLogs.groupBy(log => log.parentDir)
config.logDirs.toSet.map { logDir: String =>
val absolutePath = new File(logDir).getAbsolutePath
try {
if (!logManager.isLogDirOnline(absolutePath))
throw new KafkaStorageException(s"Log directory $absolutePath is offline")
logsByDir.get(absolutePath) match {
case Some(logs) =>
val topicInfos = logs.groupBy(_.topicPartition.topic).map{case (topic, logs) =>
new DescribeLogDirsResponseData.DescribeLogDirsTopic().setName(topic).setPartitions(
logs.filter { log =>
partitions.contains(log.topicPartition)
}.map { log =>
new DescribeLogDirsResponseData.DescribeLogDirsPartition()
.setPartitionSize(log.size)
.setPartitionIndex(log.topicPartition.partition)
.setOffsetLag(getLogEndOffsetLag(log.topicPartition, log.logEndOffset, log.isFuture))
.setIsFutureKey(log.isFuture)
}.toList.asJava)
}.toList.asJava
new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath)
.setErrorCode(Errors.NONE.code).setTopics(topicInfos)
case None =>
new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath)
.setErrorCode(Errors.NONE.code)
}
} catch {
case e: KafkaStorageException =>
warn("Unable to describe replica dirs for %s".format(absolutePath), e)
new DescribeLogDirsResponseData.DescribeLogDirsResult()
.setLogDir(absolutePath)
.setErrorCode(Errors.KAFKA_STORAGE_ERROR.code)
case t: Throwable =>
error(s"Error while describing replica in dir $absolutePath", t)
new DescribeLogDirsResponseData.DescribeLogDirsResult()
.setLogDir(absolutePath)
.setErrorCode(Errors.forException(t).code)
}
}.toList
}
def getLogEndOffsetLag(topicPartition: TopicPartition, logEndOffset: Long, isFuture: Boolean): Long = {
localLog(topicPartition) match {
case Some(log) =>
if (isFuture)
log.logEndOffset - logEndOffset
else
math.max(log.highWatermark - logEndOffset, 0)
case None =>
// return -1L to indicate that the LEO lag is not available if the replica is not created or is offline
DescribeLogDirsResponse.INVALID_OFFSET_LAG
}
}
def deleteRecords(timeout: Long,
offsetPerPartition: Map[TopicPartition, Long],
responseCallback: Map[TopicPartition, DeleteRecordsPartitionResult] => Unit): Unit = {
val timeBeforeLocalDeleteRecords = time.milliseconds
val localDeleteRecordsResults = deleteRecordsOnLocalLog(offsetPerPartition)
debug("Delete records on local log in %d ms".format(time.milliseconds - timeBeforeLocalDeleteRecords))
val deleteRecordsStatus = localDeleteRecordsResults.map { case (topicPartition, result) =>
topicPartition ->
DeleteRecordsPartitionStatus(
result.requestedOffset, // requested offset
new DeleteRecordsPartitionResult()
.setLowWatermark(result.lowWatermark)
.setErrorCode(result.error.code)
.setPartitionIndex(topicPartition.partition)) // response status
}
if (delayedDeleteRecordsRequired(localDeleteRecordsResults)) {
// create delayed delete records operation
val delayedDeleteRecords = new DelayedDeleteRecords(timeout, deleteRecordsStatus, this, responseCallback)
// create a list of (topic, partition) pairs to use as keys for this delayed delete records operation
val deleteRecordsRequestKeys = offsetPerPartition.keys.map(TopicPartitionOperationKey(_)).toSeq
// try to complete the request immediately, otherwise put it into the purgatory
// this is because while the delayed delete records operation is being created, new
// requests may arrive and hence make this operation completable.
delayedDeleteRecordsPurgatory.tryCompleteElseWatch(delayedDeleteRecords, deleteRecordsRequestKeys)
} else {
// we can respond immediately
val deleteRecordsResponseStatus = deleteRecordsStatus.map { case (k, status) => k -> status.responseStatus }
responseCallback(deleteRecordsResponseStatus)
}
}
// If all the following conditions are true, we need to put a delayed produce request and wait for replication to complete
//
// 1. required acks = -1
// 2. there is data to append
// 3. at least one partition append was successful (fewer errors than partitions)
private def delayedProduceRequestRequired(requiredAcks: Short,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
requiredAcks == -1 &&
entriesPerPartition.nonEmpty &&
localProduceResults.values.count(_.exception.isDefined) < entriesPerPartition.size
}
private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
requiredAcks == -1 || requiredAcks == 1 || requiredAcks == 0
}
/**
* Append the messages to the local replica logs
*/
private def appendToLocalLog(internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
val traceEnabled = isTraceEnabled
def processFailedRecord(topicPartition: TopicPartition, t: Throwable) = {
val logStartOffset = onlinePartition(topicPartition).map(_.logStartOffset).getOrElse(-1L)
brokerTopicStats.topicStats(topicPartition.topic).failedProduceRequestRate.mark()
brokerTopicStats.allTopicsStats.failedProduceRequestRate.mark()
error(s"Error processing append operation on partition $topicPartition", t)
logStartOffset
}
if (traceEnabled)
trace(s"Append [$entriesPerPartition] to local log")
entriesPerPartition.map { case (topicPartition, records) =>
brokerTopicStats.topicStats(topicPartition.topic).totalProduceRequestRate.mark()
brokerTopicStats.allTopicsStats.totalProduceRequestRate.mark()
// reject appending to internal topics if it is not allowed
if (Topic.isInternal(topicPartition.topic) && !internalTopicsAllowed) {
(topicPartition, LogAppendResult(
LogAppendInfo.UnknownLogAppendInfo,
Some(new InvalidTopicException(s"Cannot append to internal topic ${topicPartition.topic}"))))
} else {
try {
val partition = getPartitionOrException(topicPartition)
val info = partition.appendRecordsToLeader(records, origin, requiredAcks)
val numAppendedMessages = info.numMessages
// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
brokerTopicStats.topicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
brokerTopicStats.allTopicsStats.bytesInRate.mark(records.sizeInBytes)
brokerTopicStats.topicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
brokerTopicStats.allTopicsStats.messagesInRate.mark(numAppendedMessages)
if (traceEnabled)
trace(s"${records.sizeInBytes} written to log $topicPartition beginning at offset " +
s"${info.firstOffset.getOrElse(-1)} and ending at offset ${info.lastOffset}")
(topicPartition, LogAppendResult(info))
} catch {
// NOTE: Failed produce requests metric is not incremented for known exceptions
// it is supposed to indicate un-expected failures of a broker in handling a produce request
case e@ (_: UnknownTopicOrPartitionException |
_: NotLeaderOrFollowerException |
_: RecordTooLargeException |
_: RecordBatchTooLargeException |
_: CorruptRecordException |
_: KafkaStorageException) =>
(topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
case rve: RecordValidationException =>
val logStartOffset = processFailedRecord(topicPartition, rve.invalidException)
val recordErrors = rve.recordErrors
(topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(
logStartOffset, recordErrors, rve.invalidException.getMessage), Some(rve.invalidException)))
case t: Throwable =>
val logStartOffset = processFailedRecord(topicPartition, t)
(topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t)))
}
}
}
}
def fetchOffsetForTimestamp(topicPartition: TopicPartition,
timestamp: Long,
isolationLevel: Option[IsolationLevel],
currentLeaderEpoch: Optional[Integer],
fetchOnlyFromLeader: Boolean): Option[TimestampAndOffset] = {
val partition = getPartitionOrException(topicPartition)
partition.fetchOffsetForTimestamp(timestamp, isolationLevel, currentLeaderEpoch, fetchOnlyFromLeader)