/
KafkaStateActor.scala
1580 lines (1360 loc) · 67.2 KB
/
KafkaStateActor.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
/**
* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0
* See accompanying LICENSE file.
*/
package kafka.manager.actor.cluster
import java.io.Closeable
import java.net.InetAddress
import java.nio.ByteBuffer
import java.time.Duration
import java.util
import java.util.Properties
import java.util.concurrent.{ConcurrentLinkedDeque, TimeUnit}
import akka.actor.{ActorContext, ActorPath, ActorRef, Props}
import akka.pattern._
import com.github.benmanes.caffeine.cache.{Cache, Caffeine, RemovalCause, RemovalListener}
import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
import grizzled.slf4j.Logging
import kafka.common.OffsetAndMetadata
import kafka.manager._
import kafka.manager.base.cluster.{BaseClusterQueryActor, BaseClusterQueryCommandActor}
import kafka.manager.base.{LongRunningPoolActor, LongRunningPoolConfig}
import kafka.manager.features.{ClusterFeatures, KMDeleteTopicFeature, KMPollConsumersFeature}
import kafka.manager.model.ActorModel._
import kafka.manager.model._
import kafka.manager.utils.ZkUtils
import kafka.manager.utils.zero81.{PreferredReplicaLeaderElectionCommand, ReassignPartitionCommand}
import kafka.manager.utils.two40.{GroupMetadata, GroupMetadataKey, MemberMetadata, OffsetKey}
import org.apache.curator.framework.CuratorFramework
import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode
import org.apache.curator.framework.recipes.cache._
import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecords, KafkaConsumer}
import org.apache.kafka.common.{ConsumerGroupState, TopicPartition}
import org.apache.kafka.common.requests.DescribeGroupsResponse
import org.joda.time.{DateTime, DateTimeZone}
import scala.collection.concurrent.TrieMap
import scala.collection.immutable.Map
import scala.collection.mutable
import scala.concurrent.{ExecutionContext, Future}
import scala.util.{Failure, Success, Try}
import org.apache.kafka.clients.consumer.ConsumerConfig._
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.config.SaslConfigs
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG
import org.apache.kafka.clients.admin.{AdminClient, ConsumerGroupDescription, DescribeConsumerGroupsOptions}
import org.apache.kafka.common.KafkaFuture.BiConsumer
import org.apache.kafka.common.metrics.{KafkaMetric, MetricsReporter}
import org.apache.kafka.common.utils.Time
/**
* @author hiral
*/
import kafka.manager.utils._
import scala.collection.JavaConverters._
class NoopJMXReporter extends MetricsReporter {
override def init(metrics: util.List[KafkaMetric]): Unit = {}
override def metricChange(metric: KafkaMetric): Unit = {}
override def metricRemoval(metric: KafkaMetric): Unit = {}
override def close(): Unit = {}
override def configure(configs: util.Map[String, _]): Unit = {}
}
case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int)
case class KafkaAdminClientActorConfig(clusterContext: ClusterContext, longRunningPoolConfig: LongRunningPoolConfig, kafkaStateActorPath: ActorPath, consumerProperties: Option[Properties])
case class KafkaAdminClientActor(config: KafkaAdminClientActorConfig) extends BaseClusterQueryActor with LongRunningPoolActor {
private[this] var adminClientOption : Option[AdminClient] = None
protected implicit val clusterContext: ClusterContext = config.clusterContext
override protected def longRunningPoolConfig: LongRunningPoolConfig = config.longRunningPoolConfig
override protected def longRunningQueueFull(): Unit = {
log.error("Long running pool queue full, skipping!")
}
@scala.throws[Exception](classOf[Exception])
override def preStart() = {
super.preStart()
log.info(config.toString)
}
@scala.throws[Exception](classOf[Exception])
override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
log.error(reason, "Restarting due to [{}] when processing [{}]",
reason.getMessage, message.getOrElse(""))
super.preRestart(reason, message)
}
@scala.throws[Exception](classOf[Exception])
override def postStop(): Unit = {
log.info("Closing admin client...")
Try(adminClientOption.foreach(_.close()))
log.info("Stopped actor %s".format(self.path))
}
private def createAdminClient(bl: BrokerList): AdminClient = {
val targetBrokers : IndexedSeq[BrokerIdentity] = bl.list
val brokerListStr: String = targetBrokers.map {
b =>
val port = b.endpoints(config.clusterContext.config.securityProtocol)
s"${b.host}:$port"
}.mkString(",")
val props = new Properties()
config.consumerProperties.foreach {
cp => props.putAll(cp.asMap)
}
props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, config.clusterContext.config.securityProtocol.stringId)
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerListStr)
if(config.clusterContext.config.saslMechanism.nonEmpty){
props.put(SaslConfigs.SASL_MECHANISM, config.clusterContext.config.saslMechanism.get.stringId)
log.info(s"SASL Mechanism =${config.clusterContext.config.saslMechanism.get}")
}
if(config.clusterContext.config.jaasConfig.nonEmpty){
props.put(SaslConfigs.SASL_JAAS_CONFIG, config.clusterContext.config.jaasConfig.get)
log.info(s"SASL JAAS config=${config.clusterContext.config.jaasConfig.get}")
}
log.info(s"Creating admin client with security protocol=${config.clusterContext.config.securityProtocol.stringId} , broker list : $brokerListStr")
AdminClient.create(props)
}
override def processQueryRequest(request: QueryRequest): Unit = {
if(adminClientOption.isEmpty) {
context.actorSelection(config.kafkaStateActorPath).tell(KSGetBrokers, self)
log.error(s"AdminClient not initialized yet, cannot process request : $request")
} else {
implicit val ec = longRunningExecutionContext
request match {
case KAGetGroupSummary(groupList: Seq[String], enqueue: java.util.Queue[(String, List[MemberMetadata])]) =>
Future {
try {
adminClientOption.foreach {
client =>
val options = new DescribeConsumerGroupsOptions
options.timeoutMs(1000)
client.describeConsumerGroups(groupList.asJava, options).all().whenComplete {
(mapGroupDescription, error) => mapGroupDescription.asScala.foreach {
case (group, desc) =>
enqueue.offer(group -> desc.members().asScala.map(m => MemberMetadata.from(group, desc, m)).toList)
}
}
}
} catch {
case e: Exception =>
log.error(e, s"Failed to get group summary with admin client : $groupList")
log.error(e, s"Forcing new admin client initialization...")
Try { adminClientOption.foreach(_.close()) }
adminClientOption = None
}
}
case any: Any => log.warning("kac : processQueryRequest : Received unknown message: {}", any.toString)
}
}
}
override def processActorResponse(response: ActorResponse): Unit = {
response match {
case bl: BrokerList =>
if(bl.list.nonEmpty) {
Try {
adminClientOption = Option(createAdminClient(bl))
}.logError(s"Failed to create admin client with brokerlist : $bl")
}
case any: Any => log.warning("kac : processActorResponse : Received unknown message: {}", any.toString)
}
}
}
class KafkaAdminClient(context: => ActorContext, adminClientActorPath: ActorPath) {
def enqueueGroupMetadata(groupList: Seq[String], queue: java.util.Queue[(String, List[MemberMetadata])]) : Unit = {
Try {
context.actorSelection(adminClientActorPath).tell(KAGetGroupSummary(groupList, queue), ActorRef.noSender)
}
}
}
object KafkaManagedOffsetCache {
val supportedVersions: Set[KafkaVersion] = Set(Kafka_0_8_2_0, Kafka_0_8_2_1, Kafka_0_8_2_2, Kafka_0_9_0_0, Kafka_0_9_0_1, Kafka_0_10_0_0, Kafka_0_10_0_1, Kafka_0_10_1_0, Kafka_0_10_1_1, Kafka_0_10_2_0, Kafka_0_10_2_1, Kafka_0_11_0_0, Kafka_0_11_0_2, Kafka_1_0_0, Kafka_1_0_1, Kafka_1_1_0, Kafka_1_1_1, Kafka_2_0_0, Kafka_2_1_0, Kafka_2_1_1, Kafka_2_2_0, Kafka_2_2_1, Kafka_2_2_2, Kafka_2_3_0, Kafka_2_2_1, Kafka_2_4_0, Kafka_2_4_1, Kafka_2_5_0, Kafka_2_5_1, Kafka_2_6_0, Kafka_2_7_0, Kafka_2_8_0, Kafka_2_8_1, Kafka_3_0_0, Kafka_3_1_0, Kafka_3_1_1, Kafka_3_2_0)
val ConsumerOffsetTopic = "__consumer_offsets"
def isSupported(version: KafkaVersion) : Boolean = {
supportedVersions(version)
}
def createSet[T](): mutable.Set[T] = {
import scala.collection.JavaConverters._
java.util.Collections.newSetFromMap(
new java.util.concurrent.ConcurrentHashMap[T, java.lang.Boolean]).asScala
}
}
object KafkaManagedOffsetCacheConfig {
val defaultGroupMemberMetadataCheckMillis: Int = 30000
val defaultGroupTopicPartitionOffsetMaxSize: Int = 1000000
val defaultGroupTopicPartitionOffsetExpireDays: Int = 7
}
case class KafkaManagedOffsetCacheConfig(groupMemberMetadataCheckMillis: Int = KafkaManagedOffsetCacheConfig.defaultGroupMemberMetadataCheckMillis
, groupTopicPartitionOffsetMaxSize: Int = KafkaManagedOffsetCacheConfig.defaultGroupTopicPartitionOffsetMaxSize
, groupTopicPartitionOffsetExpireDays: Int = KafkaManagedOffsetCacheConfig.defaultGroupTopicPartitionOffsetExpireDays)
case class KafkaManagedOffsetCache(clusterContext: ClusterContext
, adminClient: KafkaAdminClient
, consumerProperties: Option[Properties]
, bootstrapBrokerList: BrokerList
, config: KafkaManagedOffsetCacheConfig
) extends Runnable with Closeable with Logging {
val groupTopicPartitionOffsetSet: mutable.Set[(String, String, Int)] = KafkaManagedOffsetCache.createSet()
val groupTopicPartitionOffsetMap:Cache[(String, String, Int), OffsetAndMetadata] = Caffeine
.newBuilder()
.maximumSize(config.groupTopicPartitionOffsetMaxSize)
.expireAfterAccess(config.groupTopicPartitionOffsetExpireDays, TimeUnit.DAYS)
.removalListener(new RemovalListener[(String, String, Int), OffsetAndMetadata] {
override def onRemoval(key: (String, String, Int), value: OffsetAndMetadata, cause: RemovalCause): Unit = {
groupTopicPartitionOffsetSet.remove(key)
}
})
.build[(String, String, Int), OffsetAndMetadata]()
val topicConsumerSetMap = new TrieMap[String, mutable.Set[String]]()
val consumerTopicSetMap = new TrieMap[String, mutable.Set[String]]()
val groupTopicPartitionMemberSet: mutable.Set[(String, String, Int)] = KafkaManagedOffsetCache.createSet()
val groupTopicPartitionMemberMap: Cache[(String, String, Int), MemberMetadata] = Caffeine
.newBuilder()
.maximumSize(config.groupTopicPartitionOffsetMaxSize)
.expireAfterAccess(config.groupTopicPartitionOffsetExpireDays, TimeUnit.DAYS)
.removalListener(new RemovalListener[(String, String, Int), MemberMetadata] {
override def onRemoval(key: (String, String, Int), value: MemberMetadata, cause: RemovalCause): Unit = {
groupTopicPartitionMemberSet.remove(key)
}
})
.build[(String, String, Int), MemberMetadata]()
private[this] val queue = new ConcurrentLinkedDeque[(String, List[MemberMetadata])]()
@volatile
private[this] var lastUpdateTimeMillis : Long = 0
private[this] var lastGroupMemberMetadataCheckMillis : Long = System.currentTimeMillis()
import KafkaManagedOffsetCache._
import kafka.manager.utils.two40.GroupMetadataManager._
require(isSupported(clusterContext.config.version), s"Kafka version not support : ${clusterContext.config}")
@volatile
private[this] var shutdown: Boolean = false
private[this] def createKafkaConsumer(): Consumer[Array[Byte], Array[Byte]] = {
val hostname = InetAddress.getLocalHost.getHostName
val brokerListStr: String = bootstrapBrokerList.list.map {
b =>
val port = b.endpoints(clusterContext.config.securityProtocol)
s"${b.host}:$port"
}.mkString(",")
val props: Properties = new Properties()
props.put(GROUP_ID_CONFIG, s"KMOffsetCache-$hostname")
props.put(BOOTSTRAP_SERVERS_CONFIG, brokerListStr)
props.put(EXCLUDE_INTERNAL_TOPICS_CONFIG, "false")
props.put(ENABLE_AUTO_COMMIT_CONFIG, "false")
props.put(KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer")
props.put(VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer")
props.put(AUTO_OFFSET_RESET_CONFIG, "latest")
props.put(METRIC_REPORTER_CLASSES_CONFIG, classOf[NoopJMXReporter].getCanonicalName)
consumerProperties.foreach {
cp => props.putAll(cp.asMap)
}
props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, clusterContext.config.securityProtocol.stringId)
if(clusterContext.config.saslMechanism.nonEmpty){
props.put(SaslConfigs.SASL_MECHANISM, clusterContext.config.saslMechanism.get.stringId)
info(s"SASL Mechanism =${clusterContext.config.saslMechanism.get}")
if(clusterContext.config.jaasConfig.nonEmpty){
props.put(SaslConfigs.SASL_JAAS_CONFIG, clusterContext.config.jaasConfig.get)
info(s"SASL JAAS config=${clusterContext.config.jaasConfig.get}")
}
}
Try {
info("Constructing new kafka consumer client using these properties: ")
props.asScala.foreach {
case (k, v) => info(s"$k=$v")
}
}
new KafkaConsumer[Array[Byte], Array[Byte]](props)
}
private[this] def performGroupMetadataCheck() : Unit = {
val currentMillis = System.currentTimeMillis()
if((lastGroupMemberMetadataCheckMillis + config.groupMemberMetadataCheckMillis) < currentMillis) {
val diff = groupTopicPartitionOffsetSet.diff(groupTopicPartitionMemberSet)
if(diff.nonEmpty) {
val groupsToBackfill = diff.map(_._1).toSeq
info(s"Backfilling group metadata for $groupsToBackfill")
adminClient.enqueueGroupMetadata(groupsToBackfill, queue)
}
lastGroupMemberMetadataCheckMillis = System.currentTimeMillis()
lastUpdateTimeMillis = System.currentTimeMillis()
}
}
private[this] def dequeueAndProcessBackFill(): Unit = {
while(!queue.isEmpty) {
val (groupId, members) = queue.pop()
members.foreach {
member =>
try {
member.assignment.foreach {
case (topic, part) =>
val k = (groupId, topic, part)
//only add it if it hasn't already been added through a new update via the offset topic
if(groupTopicPartitionMemberMap.getIfPresent(k) == null) {
groupTopicPartitionMemberMap.put(k, member)
groupTopicPartitionMemberSet.add(k)
}
}
} catch {
case e: Exception =>
error(s"Failed to get member metadata from group summary and member summary : $groupId : $member", e)
}
}
}
}
override def run(): Unit = {
if(!shutdown) {
for {
consumer <- Try {
val consumer = createKafkaConsumer()
consumer.subscribe(java.util.Arrays.asList(KafkaManagedOffsetCache.ConsumerOffsetTopic))
consumer
}.logError(s"Failed to create consumer for offset topic for cluster ${clusterContext.config.name}")
} {
try {
info(s"Consumer created for kafka offset topic consumption for cluster ${clusterContext.config.name}")
while (!shutdown) {
try {
try {
dequeueAndProcessBackFill()
performGroupMetadataCheck()
} catch {
case e: Exception =>
error("Failed to backfill group metadata", e)
}
val records: ConsumerRecords[Array[Byte], Array[Byte]] = consumer.poll(Duration.ofMillis(100))
val iterator = records.iterator()
while (iterator.hasNext) {
val record = iterator.next()
val key = record.key()
val value = record.value()
//only process records with data
if (key != null && value != null) {
readMessageKey(ByteBuffer.wrap(record.key())) match {
case OffsetKey(version, key) =>
val value: OffsetAndMetadata = readOffsetMessageValue(ByteBuffer.wrap(record.value()))
val newKey = (key.group, key.topicPartition.topic, key.topicPartition.partition)
groupTopicPartitionOffsetMap.put(newKey, value)
groupTopicPartitionOffsetSet.add(newKey)
val topic = key.topicPartition.topic
val group = key.group
val consumerSet = {
if (topicConsumerSetMap.contains(topic)) {
topicConsumerSetMap(topic)
} else {
val s = new mutable.TreeSet[String]()
topicConsumerSetMap += topic -> s
s
}
}
consumerSet += group
val topicSet = {
if (consumerTopicSetMap.contains(group)) {
consumerTopicSetMap(group)
} else {
val s = new mutable.TreeSet[String]()
consumerTopicSetMap += group -> s
s
}
}
topicSet += topic
case GroupMetadataKey(version, key) =>
val value: GroupMetadata = readGroupMessageValue(key, ByteBuffer.wrap(record.value()), Time.SYSTEM)
value.allMemberMetadata.foreach {
mm =>
mm.assignment.foreach {
case (topic, part) =>
val newKey = (key, topic, part)
groupTopicPartitionMemberMap.put(newKey, mm)
groupTopicPartitionMemberSet.add(newKey)
}
}
case other: Any =>
error(s"Unhandled key type : ${other.getClass.getCanonicalName}")
}
}
lastUpdateTimeMillis = System.currentTimeMillis()
}
} catch {
case e: Exception =>
warn(s"Failed to process a message from offset topic on cluster ${clusterContext.config.name}!", e)
}
}
} finally {
info(s"Shutting down consumer for $ConsumerOffsetTopic on cluster ${clusterContext.config.name}")
Try(consumer.close())
}
}
}
groupTopicPartitionMemberSet.clear()
groupTopicPartitionMemberMap.invalidateAll()
groupTopicPartitionMemberMap.cleanUp()
groupTopicPartitionOffsetSet.clear()
groupTopicPartitionOffsetMap.invalidateAll()
groupTopicPartitionOffsetMap.cleanUp()
info(s"KafkaManagedOffsetCache shut down for cluster ${clusterContext.config.name}")
}
def close(): Unit = {
this.shutdown = true
}
def getOffset(group: String, topic: String, part:Int) : Option[Long] = {
Option(groupTopicPartitionOffsetMap.getIfPresent((group, topic, part))).map(_.offset)
}
def getOwner(group: String, topic: String, part:Int) : Option[String] = {
Option(groupTopicPartitionMemberMap.getIfPresent((group, topic, part))).map(mm => s"${mm.memberId}:${mm.clientHost}")
}
def getConsumerTopics(group: String) : Set[String] = consumerTopicSetMap.get(group).map(_.toSet).getOrElse(Set.empty)
def getTopicConsumers(topic: String) : Set[String] = topicConsumerSetMap.get(topic).map(_.toSet).getOrElse(Set.empty)
def getConsumers : IndexedSeq[String] = consumerTopicSetMap.keys.toIndexedSeq
def getLastUpdateTimeMillis: Long = lastUpdateTimeMillis
}
case class ConsumerInstanceSubscriptions private(id: String, subs: Map[String, Int])
object ConsumerInstanceSubscriptions extends Logging {
//{"version":1,"subscription":{"DXSPreAgg":1},"pattern":"static","timestamp":"1443578242654"}
def apply(consumer: String, id: String, jsonString: String) : ConsumerInstanceSubscriptions = {
import org.json4s.jackson.JsonMethods.parse
import org.json4s.scalaz.JsonScalaz.field
val json = parse(jsonString)
val subs: Map[String, Int] = field[Map[String,Int]]("subscription")(json).fold({ e =>
error(s"[consumer=$consumer] Failed to parse consumer instance subscriptions : $id : $jsonString"); Map.empty}, identity)
new ConsumerInstanceSubscriptions(id, subs)
}
}
trait OffsetCache extends Logging {
def consumerProperties: Option[Properties]
def kafkaAdminClient: KafkaAdminClient
def clusterContext: ClusterContext
def getKafkaVersion: KafkaVersion
def getCacheTimeoutSecs: Int
def getSimpleConsumerSocketTimeoutMillis: Int
def kafkaManagedOffsetCacheConfig: KafkaManagedOffsetCacheConfig
protected[this] implicit def ec: ExecutionContext
protected[this] implicit def cf: ClusterFeatures
protected[this] val loadOffsets: Boolean
// Caches a map of partitions to offsets at a key that is the topic's name.
private[this] lazy val partitionOffsetsCache: LoadingCache[String, Future[PartitionOffsetsCapture]] = CacheBuilder.newBuilder()
.expireAfterWrite(getCacheTimeoutSecs,TimeUnit.SECONDS) // TODO - update more or less often maybe, or make it configurable
.build(
new CacheLoader[String,Future[PartitionOffsetsCapture]] {
def load(topic: String): Future[PartitionOffsetsCapture] = {
loadPartitionOffsets(topic)
}
}
)
// Get the latest offsets for the partitions of the topic,
// Code based off of the GetOffsetShell tool in kafka.tools, kafka 0.8.2.1
private[this] def loadPartitionOffsets(topic: String): Future[PartitionOffsetsCapture] = {
// Get partition leader broker information
val optPartitionsWithLeaders : Option[List[(Int, Option[BrokerIdentity])]] = getTopicPartitionLeaders(topic)
val clientId = "partitionOffsetGetter"
val time = -1
val nOffsets = 1
val simpleConsumerBufferSize = 256 * 1024
val currentActiveBrokerSet:Set[String] = getBrokerList().list.map(_.host).toSet
val partitionsByBroker = optPartitionsWithLeaders.map {
listOfPartAndBroker => listOfPartAndBroker.collect {
case (part, broker) if broker.isDefined && currentActiveBrokerSet(broker.get.host) => (broker.get, part)
}.groupBy(_._1)
}
def getKafkaConsumer() = {
new KafkaConsumer(consumerProperties.get)
}
// Get the latest offset for each partition
val futureMap: Future[PartitionOffsetsCapture] = {
partitionsByBroker.fold[Future[PartitionOffsetsCapture]]{
Future.failed(new IllegalArgumentException(s"Do not have partitions and their leaders for topic $topic"))
} { partitionsWithLeaders =>
try {
val listOfFutures = partitionsWithLeaders.toList.map(tpl => (tpl._2)).map {
case (parts) =>
val kafkaConsumer = getKafkaConsumer()
val f: Future[Map[TopicPartition, java.lang.Long]] = Future {
try {
val topicAndPartitions = parts.map(tpl => (new TopicPartition(topic, tpl._2), PartitionOffsetRequestInfo(time, nOffsets)))
val request: List[TopicPartition] = topicAndPartitions.map(f => new TopicPartition(f._1.topic(), f._1.partition()))
kafkaConsumer.endOffsets(request.asJava).asScala.toMap
} finally {
kafkaConsumer.close()
}
}
f.recover { case t =>
error(s"[topic=$topic] An error has occurred while getting topic offsets from broker $parts", t)
Map.empty[TopicPartition, java.lang.Long]
}
}
val result: Future[Map[TopicPartition, java.lang.Long]] = Future.sequence(listOfFutures).map(_.foldRight(Map.empty[TopicPartition, java.lang.Long])((b, a) => b ++ a))
result.map(m => PartitionOffsetsCapture(System.currentTimeMillis(), m.map(f => (f._1.partition(), f._2.toLong))))
}
catch {
case e: Exception =>
error(s"Failed to get offsets for topic $topic", e)
Future.failed(e)
}
}
}
futureMap.failed.foreach {
t => error(s"[topic=$topic] An error has occurred while getting topic offsets", t)
}
futureMap
}
private[this] def emptyPartitionOffsetsCapture: Future[PartitionOffsetsCapture] = Future.successful(PartitionOffsetsCapture(System.currentTimeMillis(), Map()))
protected def getTopicPartitionLeaders(topic: String) : Option[List[(Int, Option[BrokerIdentity])]]
protected def getTopicDescription(topic: String, interactive: Boolean) : Option[TopicDescription]
protected def getBrokerList : () => BrokerList
protected def readConsumerOffsetByTopicPartition(consumer: String, topic: String, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, Long]
protected def readConsumerOwnerByTopicPartition(consumer: String, topic: String, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, String]
protected def getConsumerTopicsFromIds(consumer: String) : Set[String]
protected def getConsumerTopicsFromOffsets(consumer: String) : Set[String]
protected def getConsumerTopicsFromOwners(consumer: String) : Set[String]
protected def getZKManagedConsumerList: IndexedSeq[ConsumerNameAndType]
protected def lastUpdateMillisZK : Long
protected def getConsumerTopics(consumer: String) : Set[String] = {
getConsumerTopicsFromOffsets(consumer) ++ getConsumerTopicsFromOwners(consumer) ++ getConsumerTopicsFromIds(consumer)
}
private[this] var kafkaManagedOffsetCache : Option[KafkaManagedOffsetCache] = None
private[this] lazy val hasNonSecureEndpoint = getBrokerList().list.exists(_.nonSecure)
def start() : Unit = {
if(KafkaManagedOffsetCache.isSupported(clusterContext.config.version)) {
if(kafkaManagedOffsetCache.isEmpty) {
info("Starting kafka managed offset cache ...")
Try {
val bl = getBrokerList()
require(bl.list.nonEmpty, "Cannot consume from offset topic when there are no brokers!")
val of = new KafkaManagedOffsetCache(clusterContext, kafkaAdminClient, consumerProperties, bl, kafkaManagedOffsetCacheConfig)
kafkaManagedOffsetCache = Option(of)
val t = new Thread(of, "KafkaManagedOffsetCache")
t.start()
}
}
} else {
throw new IllegalArgumentException(s"Unsupported Kafka Version: ${clusterContext.config.version}")
}
}
def stop() : Unit = {
kafkaManagedOffsetCache.foreach { of =>
info("Stopping kafka managed offset cache ...")
Try {
of.close()
}
}
}
def getTopicPartitionOffsets(topic: String, interactive: Boolean) : Future[PartitionOffsetsCapture] = {
if((interactive || loadOffsets) && hasNonSecureEndpoint) {
partitionOffsetsCache.get(topic)
} else {
emptyPartitionOffsetsCapture
}
}
protected def readKafkaManagedConsumerOffsetByTopicPartition(consumer: String
, topic: String
, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, Long] = {
kafkaManagedOffsetCache.fold(Map.empty[Int,Long]) {
oc =>
tpi.map {
case (part, _) =>
part -> oc.getOffset(consumer, topic, part).getOrElse(-1L)
}
}
}
protected def readKafkaManagedConsumerOwnerByTopicPartition(consumer: String
, topic: String
, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, String] = {
kafkaManagedOffsetCache.fold(Map.empty[Int,String]) {
oc =>
tpi.map {
case (part, _) =>
part -> oc.getOwner(consumer, topic, part).getOrElse("")
}
}
}
protected def getKafkaManagedConsumerTopics(consumer: String) : Set[String] = {
kafkaManagedOffsetCache.fold(Set.empty[String]) {
oc => oc.getConsumerTopics(consumer)
}
}
protected def getKafkaManagedConsumerList : IndexedSeq[ConsumerNameAndType] = {
kafkaManagedOffsetCache.fold(IndexedSeq.empty[ConsumerNameAndType]) {
oc => oc.getConsumers.map(name => ConsumerNameAndType(name, KafkaManagedConsumer))
}
}
final def lastUpdateMillis : Long = {
Math.max(lastUpdateMillisZK, kafkaManagedOffsetCache.map(_.getLastUpdateTimeMillis).getOrElse(Long.MinValue))
}
final def getConsumerDescription(consumer: String, consumerType: ConsumerType) : ConsumerDescription = {
val consumerTopics: Set[String] = getKafkaVersion match {
case Kafka_0_8_1_1 => getConsumerTopicsFromOffsets(consumer)
case _ =>
consumerType match {
case ZKManagedConsumer =>
getConsumerTopicsFromOffsets(consumer) ++ getConsumerTopicsFromOwners(consumer)
case KafkaManagedConsumer =>
getKafkaManagedConsumerTopics(consumer)
}
}
val topicDescriptions: Map[String, ConsumedTopicDescription] = consumerTopics.map { topic =>
val topicDesc = getConsumedTopicDescription(consumer, topic, false, consumerType)
(topic, topicDesc)
}.toMap
ConsumerDescription(consumer, topicDescriptions, consumerType)
}
final def getConsumedTopicDescription(consumer:String
, topic:String
, interactive: Boolean
, consumerType: ConsumerType) : ConsumedTopicDescription = {
val optTopic = getTopicDescription(topic, interactive)
val optTpi = optTopic.map(TopicIdentity.getTopicPartitionIdentity(_, None))
val (partitionOffsets, partitionOwners) = consumerType match {
case ZKManagedConsumer =>
val partitionOffsets = for {
td <- optTopic
tpi <- optTpi
} yield {
readConsumerOffsetByTopicPartition(consumer, topic, tpi)
}
val partitionOwners = for {
td <- optTopic
tpi <- optTpi
} yield {
readConsumerOwnerByTopicPartition(consumer, topic, tpi)
}
(partitionOffsets, partitionOwners)
case KafkaManagedConsumer =>
val partitionOffsets = for {
td <- optTopic
tpi <- optTpi
} yield {
readKafkaManagedConsumerOffsetByTopicPartition(consumer, topic, tpi)
}
val partitionOwners = for {
td <- optTopic
tpi <- optTpi
} yield {
readKafkaManagedConsumerOwnerByTopicPartition(consumer, topic, tpi)
}
(partitionOffsets, partitionOwners)
}
val numPartitions: Int = math.max(optTopic.flatMap(_.partitionState.map(_.size)).getOrElse(0),
partitionOffsets.map(_.size).getOrElse(0))
ConsumedTopicDescription(consumer, topic, numPartitions, optTopic, partitionOwners, partitionOffsets)
}
final def getConsumerList: ConsumerList = {
ConsumerList(getKafkaManagedConsumerList ++ getZKManagedConsumerList, clusterContext)
}
}
case class OffsetCacheActive(curator: CuratorFramework
, kafkaAdminClient: KafkaAdminClient
, clusterContext: ClusterContext
, partitionLeaders: String => Option[List[(Int, Option[BrokerIdentity])]]
, topicDescriptions: (String, Boolean) => Option[TopicDescription]
, cacheTimeoutSecs: Int
, socketTimeoutMillis: Int
, kafkaVersion: KafkaVersion
, consumerProperties: Option[Properties]
, kafkaManagedOffsetCacheConfig: KafkaManagedOffsetCacheConfig
, getBrokerList : () => BrokerList
)
(implicit protected[this] val ec: ExecutionContext, val cf: ClusterFeatures) extends OffsetCache {
def getKafkaVersion: KafkaVersion = kafkaVersion
def getCacheTimeoutSecs: Int = cacheTimeoutSecs
def getSimpleConsumerSocketTimeoutMillis: Int = socketTimeoutMillis
val loadOffsets = featureGateFold(KMPollConsumersFeature)(false, true)
private[this] val consumersTreeCacheListener = new TreeCacheListener {
override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = {
event.getType match {
case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED |
TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED =>
consumersTreeCacheLastUpdateMillis = System.currentTimeMillis()
case _ =>
//do nothing
}
}
}
private[this] val consumersTreeCache = new TreeCache(curator, ZkUtils.ConsumersPath)
@volatile
private[this] var consumersTreeCacheLastUpdateMillis : Long = System.currentTimeMillis()
private[this] def withConsumersTreeCache[T](fn: TreeCache => T) : Option[T] = {
Option(fn(consumersTreeCache))
}
protected def getTopicPartitionLeaders(topic: String) : Option[List[(Int, Option[BrokerIdentity])]] = partitionLeaders(topic)
protected def getTopicDescription(topic: String, interactive: Boolean) : Option[TopicDescription] = topicDescriptions(topic, interactive)
override def start(): Unit = {
super.start()
info("Starting consumers tree cache...")
consumersTreeCache.start()
info("Adding consumers tree cache listener...")
consumersTreeCache.getListenable.addListener(consumersTreeCacheListener)
}
override def stop(): Unit = {
super.stop()
info("Removing consumers tree cache listener...")
Try(consumersTreeCache.getListenable.removeListener(consumersTreeCacheListener))
info("Shutting down consumers tree cache...")
Try(consumersTreeCache.close())
}
protected def lastUpdateMillisZK : Long = consumersTreeCacheLastUpdateMillis
protected def readConsumerOffsetByTopicPartition(consumer: String, topic: String, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, Long] = {
tpi.map {
case (p, _) =>
val offsetPath = "%s/%s/%s/%s/%s".format(ZkUtils.ConsumersPath, consumer, "offsets", topic, p)
(p, Option(consumersTreeCache.getCurrentData(offsetPath)).flatMap(cd => Option(cd.getData)).map(asString).getOrElse("-1").toLong)
}
}
protected def readConsumerOwnerByTopicPartition(consumer: String, topic: String, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, String] = {
tpi.map {
case (p, _) =>
val offsetPath = "%s/%s/%s/%s/%s".format(ZkUtils.ConsumersPath, consumer, "owners", topic, p)
(p, Option(consumersTreeCache.getCurrentData(offsetPath)).flatMap(cd => Option(cd.getData)).map(asString).getOrElse(""))
}
}
protected def getConsumerTopicsFromIds(consumer: String) : Set[String] = {
val zkPath = "%s/%s/%s".format(ZkUtils.ConsumersPath,consumer,"ids")
Option(consumersTreeCache.getCurrentChildren(zkPath)).map(_.asScala.toMap.map {
case (id, cd) => ConsumerInstanceSubscriptions.apply(consumer, id, Option(cd).map(_.getData).map(asString).getOrElse("{}"))
}.map(_.subs.keys).flatten.toSet).getOrElse(Set.empty)
}
protected def getConsumerTopicsFromOffsets(consumer: String) : Set[String] = {
val zkPath = "%s/%s/%s".format(ZkUtils.ConsumersPath,consumer,"offsets")
Option(consumersTreeCache.getCurrentChildren(zkPath)).map(_.asScala.toMap.keySet).getOrElse(Set.empty)
}
protected def getConsumerTopicsFromOwners(consumer: String) : Set[String] = {
val zkPath = "%s/%s/%s".format(ZkUtils.ConsumersPath,consumer,"owners")
Option(consumersTreeCache.getCurrentChildren(zkPath)).map(_.asScala.toMap.keySet).getOrElse(Set.empty)
}
protected def getZKManagedConsumerList: IndexedSeq[ConsumerNameAndType] = {
withConsumersTreeCache { cache =>
cache.getCurrentChildren(ZkUtils.ConsumersPath)
}.fold {
IndexedSeq.empty[ConsumerNameAndType]
} { data: java.util.Map[String, ChildData] =>
data.asScala.filter{
case (consumer, childData) =>
if (clusterContext.config.filterConsumers)
// Defining "inactive consumer" as a consumer that is missing one of three children ids/ offsets/ or owners/
childData.getStat.getNumChildren > 2
else true
}.keySet.toIndexedSeq.map(name => ConsumerNameAndType(name, ZKManagedConsumer))
}
}
}
case class OffsetCachePassive(curator: CuratorFramework
, kafkaAdminClient: KafkaAdminClient
, clusterContext: ClusterContext
, partitionLeaders: String => Option[List[(Int, Option[BrokerIdentity])]]
, topicDescriptions: (String, Boolean) => Option[TopicDescription]
, cacheTimeoutSecs: Int
, socketTimeoutMillis: Int
, kafkaVersion: KafkaVersion
, consumerProperties: Option[Properties]
, kafkaManagedOffsetCacheConfig: KafkaManagedOffsetCacheConfig
, getBrokerList : () => BrokerList
)
(implicit protected[this] val ec: ExecutionContext, val cf: ClusterFeatures) extends OffsetCache {
def getKafkaVersion: KafkaVersion = kafkaVersion
def getCacheTimeoutSecs: Int = cacheTimeoutSecs
def getSimpleConsumerSocketTimeoutMillis: Int = socketTimeoutMillis
val loadOffsets = featureGateFold(KMPollConsumersFeature)(false, true)
private[this] val consumersPathChildrenCacheListener = new PathChildrenCacheListener {
override def childEvent(client: CuratorFramework, event: PathChildrenCacheEvent): Unit = {
event.getType match {
case PathChildrenCacheEvent.Type.INITIALIZED | PathChildrenCacheEvent.Type.CHILD_ADDED |
PathChildrenCacheEvent.Type.CHILD_REMOVED | PathChildrenCacheEvent.Type.CHILD_UPDATED =>
consumersTreeCacheLastUpdateMillis = System.currentTimeMillis()
case _ =>
//do nothing
}
}
}
private[this] val consumersPathChildrenCache = new PathChildrenCache(curator, ZkUtils.ConsumersPath, true)
@volatile
private[this] var consumersTreeCacheLastUpdateMillis : Long = System.currentTimeMillis()
private[this] def withConsumersPathChildrenCache[T](fn: PathChildrenCache => T) : Option[T] = {
Option(fn(consumersPathChildrenCache))
}
protected def getTopicPartitionLeaders(topic: String) : Option[List[(Int, Option[BrokerIdentity])]] = partitionLeaders(topic)
protected def getTopicDescription(topic: String, interactive: Boolean) : Option[TopicDescription] = topicDescriptions(topic, interactive)
override def start(): Unit = {
super.start()
info("Starting consumers path children cache...")
consumersPathChildrenCache.start(StartMode.BUILD_INITIAL_CACHE)
info("Adding consumers path children cache listener...")
consumersPathChildrenCache.getListenable.addListener(consumersPathChildrenCacheListener)
}
override def stop(): Unit = {
super.stop()
info("Removing consumers path children cache listener...")
Try(consumersPathChildrenCache.getListenable.removeListener(consumersPathChildrenCacheListener))
info("Shutting down consumers path children cache...")
Try(consumersPathChildrenCache.close())
}
protected def lastUpdateMillisZK : Long = consumersTreeCacheLastUpdateMillis
protected def readConsumerOffsetByTopicPartition(consumer: String, topic: String, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, Long] = {
tpi.map {
case (p, _) =>
val offsetPath = "%s/%s/%s/%s/%s".format(ZkUtils.ConsumersPath, consumer, "offsets", topic, p)
(p, ZkUtils.readDataMaybeNull(curator, offsetPath)._1.map(_.toLong).getOrElse(-1L))
}
}
protected def readConsumerOwnerByTopicPartition(consumer: String, topic: String, tpi: Map[Int, TopicPartitionIdentity]) : Map[Int, String] = {
tpi.map {
case (p, _) =>
val ownerPath = "%s/%s/%s/%s/%s".format(ZkUtils.ConsumersPath, consumer, "owners", topic, p)
(p, ZkUtils.readDataMaybeNull(curator, ownerPath)._1.orNull)
}.filter(_._2 != null)
}
protected def getConsumerTopicsFromIds(consumer: String) : Set[String] = {
val zkPath = "%s/%s/%s".format(ZkUtils.ConsumersPath,consumer,"ids")
val ids = Try(Option(curator.getChildren.forPath(zkPath)).map(_.asScala.toIterable)).toOption.flatten.getOrElse(Iterable.empty)
val topicList : Iterable[Iterable[String]] = for {
id <- ids
idPath = "%s/%s".format(zkPath, id)
} yield {
ZkUtils.readDataMaybeNull(
curator, idPath)._1.map(ConsumerInstanceSubscriptions.apply(consumer, id, _)).map(_.subs.keys).getOrElse(Iterable.empty)
}
topicList.flatten.toSet
}
protected def getConsumerTopicsFromOffsets(consumer: String) : Set[String] = {
val zkPath = "%s/%s/%s".format(ZkUtils.ConsumersPath,consumer,"offsets")
Try(Option(curator.getChildren.forPath(zkPath)).map(_.asScala.toSet)).toOption.flatten.getOrElse(Set.empty)
}
protected def getConsumerTopicsFromOwners(consumer: String) : Set[String] = {
val zkPath = "%s/%s/%s".format(ZkUtils.ConsumersPath,consumer,"owners")
Try(Option(curator.getChildren.forPath(zkPath)).map(_.asScala.toSet)).toOption.flatten.getOrElse(Set.empty)
}
protected def getZKManagedConsumerList: IndexedSeq[ConsumerNameAndType] = {
withConsumersPathChildrenCache { cache =>
val currentData = cache.getCurrentData
currentData
}.fold {
IndexedSeq.empty[ConsumerNameAndType]
} { data: java.util.List[ChildData] =>
data.asScala.map(cd => ConsumerNameAndType(cd.getPath.split("/").last, ZKManagedConsumer)).toIndexedSeq
}
}
}
case class KafkaStateActorConfig(curator: CuratorFramework
, pinnedDispatcherName: String
, clusterContext: ClusterContext
, offsetCachePoolConfig: LongRunningPoolConfig
, kafkaAdminClientPoolConfig: LongRunningPoolConfig
, partitionOffsetCacheTimeoutSecs: Int
, simpleConsumerSocketTimeoutMillis: Int
, consumerProperties: Option[Properties]
, kafkaManagedOffsetCacheConfig: KafkaManagedOffsetCacheConfig
)
class KafkaStateActor(config: KafkaStateActorConfig) extends BaseClusterQueryCommandActor with LongRunningPoolActor {
protected implicit val clusterContext: ClusterContext = config.clusterContext
protected implicit val cf: ClusterFeatures = clusterContext.clusterFeatures
override protected def longRunningPoolConfig: LongRunningPoolConfig = config.offsetCachePoolConfig
override protected def longRunningQueueFull(): Unit = {
log.error("Long running pool queue full, skipping!")
}
private[this] val kaConfig = KafkaAdminClientActorConfig(
clusterContext,
config.kafkaAdminClientPoolConfig,
self.path,
config.consumerProperties
)
private[this] val kaProps = Props(classOf[KafkaAdminClientActor],kaConfig)
private[this] val kafkaAdminClientActor : ActorPath = context.actorOf(kaProps.withDispatcher(config.pinnedDispatcherName),"kafka-admin-client").path
private[this] val kafkaAdminClient = new KafkaAdminClient(context, kafkaAdminClientActor)
// e.g. /brokers/topics/analytics_content/partitions/0/state
private[this] val topicsTreeCache = new TreeCache(config.curator,ZkUtils.BrokerTopicsPath)
private[this] val topicsConfigPathCache = new PathChildrenCache(config.curator,ZkUtils.TopicConfigPath,true)
private[this] val brokerConfigPathCache = new PathChildrenCache(config.curator,ZkUtils.BrokerConfigPath,true)