blob: 581cf497398c9ec176530385747ffa17c6d411e2 [file] [log] [blame]
Takahiro Suzuki241c10e2020-12-17 20:17:57 +09001/*
2 * Copyright 2018-present Open Networking Foundation
3
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at
7
8 * http://www.apache.org/licenses/LICENSE-2.0
9
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
15 */
16package kafka
17
18import (
19 "context"
20 "errors"
21 "fmt"
22 "strings"
23 "sync"
24 "time"
25
26 "github.com/Shopify/sarama"
27 scc "github.com/bsm/sarama-cluster"
28 "github.com/eapache/go-resiliency/breaker"
29 "github.com/golang/protobuf/proto"
30 "github.com/golang/protobuf/ptypes"
31 "github.com/google/uuid"
32 "github.com/opencord/voltha-lib-go/v3/pkg/log"
33 ic "github.com/opencord/voltha-protos/v3/go/inter_container"
34)
35
36// consumerChannels represents one or more consumers listening on a kafka topic. Once a message is received on that
37// topic, the consumer(s) broadcasts the message to all the listening channels. The consumer can be a partition
38//consumer or a group consumer
39type consumerChannels struct {
40 consumers []interface{}
41 channels []chan *ic.InterContainerMessage
42}
43
44// static check to ensure SaramaClient implements Client
45var _ Client = &SaramaClient{}
46
47// SaramaClient represents the messaging proxy
48type SaramaClient struct {
49 cAdmin sarama.ClusterAdmin
50 KafkaAddress string
51 producer sarama.AsyncProducer
52 consumer sarama.Consumer
53 groupConsumers map[string]*scc.Consumer
54 lockOfGroupConsumers sync.RWMutex
55 consumerGroupPrefix string
56 consumerType int
57 consumerGroupName string
58 producerFlushFrequency int
59 producerFlushMessages int
60 producerFlushMaxmessages int
61 producerRetryMax int
62 producerRetryBackOff time.Duration
63 producerReturnSuccess bool
64 producerReturnErrors bool
65 consumerMaxwait int
66 maxProcessingTime int
67 numPartitions int
68 numReplicas int
69 autoCreateTopic bool
70 doneCh chan int
71 metadataCallback func(fromTopic string, timestamp time.Time)
72 topicToConsumerChannelMap map[string]*consumerChannels
73 lockTopicToConsumerChannelMap sync.RWMutex
74 topicLockMap map[string]*sync.RWMutex
75 lockOfTopicLockMap sync.RWMutex
76 metadataMaxRetry int
77 alive bool
78 liveness chan bool
79 livenessChannelInterval time.Duration
80 lastLivenessTime time.Time
81 started bool
82 healthy bool
83 healthiness chan bool
84}
85
86type SaramaClientOption func(*SaramaClient)
87
88func Address(address string) SaramaClientOption {
89 return func(args *SaramaClient) {
90 args.KafkaAddress = address
91 }
92}
93
94func ConsumerGroupPrefix(prefix string) SaramaClientOption {
95 return func(args *SaramaClient) {
96 args.consumerGroupPrefix = prefix
97 }
98}
99
100func ConsumerGroupName(name string) SaramaClientOption {
101 return func(args *SaramaClient) {
102 args.consumerGroupName = name
103 }
104}
105
106func ConsumerType(consumer int) SaramaClientOption {
107 return func(args *SaramaClient) {
108 args.consumerType = consumer
109 }
110}
111
112func ProducerFlushFrequency(frequency int) SaramaClientOption {
113 return func(args *SaramaClient) {
114 args.producerFlushFrequency = frequency
115 }
116}
117
118func ProducerFlushMessages(num int) SaramaClientOption {
119 return func(args *SaramaClient) {
120 args.producerFlushMessages = num
121 }
122}
123
124func ProducerFlushMaxMessages(num int) SaramaClientOption {
125 return func(args *SaramaClient) {
126 args.producerFlushMaxmessages = num
127 }
128}
129
130func ProducerMaxRetries(num int) SaramaClientOption {
131 return func(args *SaramaClient) {
132 args.producerRetryMax = num
133 }
134}
135
136func ProducerRetryBackoff(duration time.Duration) SaramaClientOption {
137 return func(args *SaramaClient) {
138 args.producerRetryBackOff = duration
139 }
140}
141
142func ProducerReturnOnErrors(opt bool) SaramaClientOption {
143 return func(args *SaramaClient) {
144 args.producerReturnErrors = opt
145 }
146}
147
148func ProducerReturnOnSuccess(opt bool) SaramaClientOption {
149 return func(args *SaramaClient) {
150 args.producerReturnSuccess = opt
151 }
152}
153
154func ConsumerMaxWait(wait int) SaramaClientOption {
155 return func(args *SaramaClient) {
156 args.consumerMaxwait = wait
157 }
158}
159
160func MaxProcessingTime(pTime int) SaramaClientOption {
161 return func(args *SaramaClient) {
162 args.maxProcessingTime = pTime
163 }
164}
165
166func NumPartitions(number int) SaramaClientOption {
167 return func(args *SaramaClient) {
168 args.numPartitions = number
169 }
170}
171
172func NumReplicas(number int) SaramaClientOption {
173 return func(args *SaramaClient) {
174 args.numReplicas = number
175 }
176}
177
178func AutoCreateTopic(opt bool) SaramaClientOption {
179 return func(args *SaramaClient) {
180 args.autoCreateTopic = opt
181 }
182}
183
184func MetadatMaxRetries(retry int) SaramaClientOption {
185 return func(args *SaramaClient) {
186 args.metadataMaxRetry = retry
187 }
188}
189
190func LivenessChannelInterval(opt time.Duration) SaramaClientOption {
191 return func(args *SaramaClient) {
192 args.livenessChannelInterval = opt
193 }
194}
195
196func NewSaramaClient(opts ...SaramaClientOption) *SaramaClient {
197 client := &SaramaClient{
198 KafkaAddress: DefaultKafkaAddress,
199 }
200 client.consumerType = DefaultConsumerType
201 client.producerFlushFrequency = DefaultProducerFlushFrequency
202 client.producerFlushMessages = DefaultProducerFlushMessages
203 client.producerFlushMaxmessages = DefaultProducerFlushMaxmessages
204 client.producerReturnErrors = DefaultProducerReturnErrors
205 client.producerReturnSuccess = DefaultProducerReturnSuccess
206 client.producerRetryMax = DefaultProducerRetryMax
207 client.producerRetryBackOff = DefaultProducerRetryBackoff
208 client.consumerMaxwait = DefaultConsumerMaxwait
209 client.maxProcessingTime = DefaultMaxProcessingTime
210 client.numPartitions = DefaultNumberPartitions
211 client.numReplicas = DefaultNumberReplicas
212 client.autoCreateTopic = DefaultAutoCreateTopic
213 client.metadataMaxRetry = DefaultMetadataMaxRetry
214 client.livenessChannelInterval = DefaultLivenessChannelInterval
215
216 for _, option := range opts {
217 option(client)
218 }
219
220 client.groupConsumers = make(map[string]*scc.Consumer)
221
222 client.lockTopicToConsumerChannelMap = sync.RWMutex{}
223 client.topicLockMap = make(map[string]*sync.RWMutex)
224 client.lockOfTopicLockMap = sync.RWMutex{}
225 client.lockOfGroupConsumers = sync.RWMutex{}
226
227 // healthy and alive until proven otherwise
228 client.alive = true
229 client.healthy = true
230
231 return client
232}
233
234func (sc *SaramaClient) Start() error {
235 logger.Info("Starting-kafka-sarama-client")
236
237 // Create the Done channel
238 sc.doneCh = make(chan int, 1)
239
240 var err error
241
242 // Add a cleanup in case of failure to startup
243 defer func() {
244 if err != nil {
245 sc.Stop()
246 }
247 }()
248
249 // Create the Cluster Admin
250 if err = sc.createClusterAdmin(); err != nil {
251 logger.Errorw("Cannot-create-cluster-admin", log.Fields{"error": err})
252 return err
253 }
254
255 // Create the Publisher
256 if err := sc.createPublisher(); err != nil {
257 logger.Errorw("Cannot-create-kafka-publisher", log.Fields{"error": err})
258 return err
259 }
260
261 if sc.consumerType == DefaultConsumerType {
262 // Create the master consumers
263 if err := sc.createConsumer(); err != nil {
264 logger.Errorw("Cannot-create-kafka-consumers", log.Fields{"error": err})
265 return err
266 }
267 }
268
269 // Create the topic to consumers/channel map
270 sc.topicToConsumerChannelMap = make(map[string]*consumerChannels)
271
272 logger.Info("kafka-sarama-client-started")
273
274 sc.started = true
275
276 return nil
277}
278
279func (sc *SaramaClient) Stop() {
280 logger.Info("stopping-sarama-client")
281
282 sc.started = false
283
284 //Send a message over the done channel to close all long running routines
285 sc.doneCh <- 1
286
287 if sc.producer != nil {
288 if err := sc.producer.Close(); err != nil {
289 logger.Errorw("closing-producer-failed", log.Fields{"error": err})
290 }
291 }
292
293 if sc.consumer != nil {
294 if err := sc.consumer.Close(); err != nil {
295 logger.Errorw("closing-partition-consumer-failed", log.Fields{"error": err})
296 }
297 }
298
299 for key, val := range sc.groupConsumers {
300 logger.Debugw("closing-group-consumer", log.Fields{"topic": key})
301 if err := val.Close(); err != nil {
302 logger.Errorw("closing-group-consumer-failed", log.Fields{"error": err, "topic": key})
303 }
304 }
305
306 if sc.cAdmin != nil {
307 if err := sc.cAdmin.Close(); err != nil {
308 logger.Errorw("closing-cluster-admin-failed", log.Fields{"error": err})
309 }
310 }
311
312 //TODO: Clear the consumers map
313 //sc.clearConsumerChannelMap()
314
315 logger.Info("sarama-client-stopped")
316}
317
318//createTopic is an internal function to create a topic on the Kafka Broker. No locking is required as
319// the invoking function must hold the lock
320func (sc *SaramaClient) createTopic(topic *Topic, numPartition int, repFactor int) error {
321 // Set the topic details
322 topicDetail := &sarama.TopicDetail{}
323 topicDetail.NumPartitions = int32(numPartition)
324 topicDetail.ReplicationFactor = int16(repFactor)
325 topicDetail.ConfigEntries = make(map[string]*string)
326 topicDetails := make(map[string]*sarama.TopicDetail)
327 topicDetails[topic.Name] = topicDetail
328
329 if err := sc.cAdmin.CreateTopic(topic.Name, topicDetail, false); err != nil {
330 if err == sarama.ErrTopicAlreadyExists {
331 // Not an error
332 logger.Debugw("topic-already-exist", log.Fields{"topic": topic.Name})
333 return nil
334 }
335 logger.Errorw("create-topic-failure", log.Fields{"error": err})
336 return err
337 }
338 // TODO: Wait until the topic has been created. No API is available in the Sarama clusterAdmin to
339 // do so.
340 logger.Debugw("topic-created", log.Fields{"topic": topic, "numPartition": numPartition, "replicationFactor": repFactor})
341 return nil
342}
343
344//CreateTopic is a public API to create a topic on the Kafka Broker. It uses a lock on a specific topic to
345// ensure no two go routines are performing operations on the same topic
346func (sc *SaramaClient) CreateTopic(topic *Topic, numPartition int, repFactor int) error {
347 sc.lockTopic(topic)
348 defer sc.unLockTopic(topic)
349
350 return sc.createTopic(topic, numPartition, repFactor)
351}
352
353//DeleteTopic removes a topic from the kafka Broker
354func (sc *SaramaClient) DeleteTopic(topic *Topic) error {
355 sc.lockTopic(topic)
356 defer sc.unLockTopic(topic)
357
358 // Remove the topic from the broker
359 if err := sc.cAdmin.DeleteTopic(topic.Name); err != nil {
360 if err == sarama.ErrUnknownTopicOrPartition {
361 // Not an error as does not exist
362 logger.Debugw("topic-not-exist", log.Fields{"topic": topic.Name})
363 return nil
364 }
365 logger.Errorw("delete-topic-failed", log.Fields{"topic": topic, "error": err})
366 return err
367 }
368
369 // Clear the topic from the consumer channel. This will also close any consumers listening on that topic.
370 if err := sc.clearTopicFromConsumerChannelMap(*topic); err != nil {
371 logger.Errorw("failure-clearing-channels", log.Fields{"topic": topic, "error": err})
372 return err
373 }
374 return nil
375}
376
377// Subscribe registers a caller to a topic. It returns a channel that the caller can use to receive
378// messages from that topic
379func (sc *SaramaClient) Subscribe(topic *Topic, kvArgs ...*KVArg) (<-chan *ic.InterContainerMessage, error) {
380 sc.lockTopic(topic)
381 defer sc.unLockTopic(topic)
382
383 logger.Debugw("subscribe", log.Fields{"topic": topic.Name})
384
385 // If a consumers already exist for that topic then resuse it
386 if consumerCh := sc.getConsumerChannel(topic); consumerCh != nil {
387 logger.Debugw("topic-already-subscribed", log.Fields{"topic": topic.Name})
388 // Create a channel specific for that consumers and add it to the consumers channel map
389 ch := make(chan *ic.InterContainerMessage)
390 sc.addChannelToConsumerChannelMap(topic, ch)
391 return ch, nil
392 }
393
394 // Register for the topic and set it up
395 var consumerListeningChannel chan *ic.InterContainerMessage
396 var err error
397
398 // Use the consumerType option to figure out the type of consumer to launch
399 if sc.consumerType == PartitionConsumer {
400 if sc.autoCreateTopic {
401 if err = sc.createTopic(topic, sc.numPartitions, sc.numReplicas); err != nil {
402 logger.Errorw("create-topic-failure", log.Fields{"error": err, "topic": topic.Name})
403 return nil, err
404 }
405 }
406 if consumerListeningChannel, err = sc.setupPartitionConsumerChannel(topic, getOffset(kvArgs...)); err != nil {
407 logger.Warnw("create-consumers-channel-failure", log.Fields{"error": err, "topic": topic.Name})
408 return nil, err
409 }
410 } else if sc.consumerType == GroupCustomer {
411 // TODO: create topic if auto create is on. There is an issue with the sarama cluster library that
412 // does not consume from a precreated topic in some scenarios
413 //if sc.autoCreateTopic {
414 // if err = sc.createTopic(topic, sc.numPartitions, sc.numReplicas); err != nil {
415 // logger.Errorw("create-topic-failure", logger.Fields{"error": err, "topic": topic.Name})
416 // return nil, err
417 // }
418 //}
419 //groupId := sc.consumerGroupName
420 groupId := getGroupId(kvArgs...)
421 // Include the group prefix
422 if groupId != "" {
423 groupId = sc.consumerGroupPrefix + groupId
424 } else {
425 // Need to use a unique group Id per topic
426 groupId = sc.consumerGroupPrefix + topic.Name
427 }
428 if consumerListeningChannel, err = sc.setupGroupConsumerChannel(topic, groupId, getOffset(kvArgs...)); err != nil {
429 logger.Warnw("create-consumers-channel-failure", log.Fields{"error": err, "topic": topic.Name, "groupId": groupId})
430 return nil, err
431 }
432
433 } else {
434 logger.Warnw("unknown-consumer-type", log.Fields{"consumer-type": sc.consumerType})
435 return nil, errors.New("unknown-consumer-type")
436 }
437
438 return consumerListeningChannel, nil
439}
440
441//UnSubscribe unsubscribe a consumer from a given topic
442func (sc *SaramaClient) UnSubscribe(topic *Topic, ch <-chan *ic.InterContainerMessage) error {
443 sc.lockTopic(topic)
444 defer sc.unLockTopic(topic)
445
446 logger.Debugw("unsubscribing-channel-from-topic", log.Fields{"topic": topic.Name})
447 var err error
448 if err = sc.removeChannelFromConsumerChannelMap(*topic, ch); err != nil {
449 logger.Errorw("failed-removing-channel", log.Fields{"error": err})
450 }
451 if err = sc.deleteFromGroupConsumers(topic.Name); err != nil {
452 logger.Errorw("failed-deleting-group-consumer", log.Fields{"error": err})
453 }
454 return err
455}
456
457func (sc *SaramaClient) SubscribeForMetadata(callback func(fromTopic string, timestamp time.Time)) {
458 sc.metadataCallback = callback
459}
460
461func (sc *SaramaClient) updateLiveness(alive bool) {
462 // Post a consistent stream of liveness data to the channel,
463 // so that in a live state, the core does not timeout and
464 // send a forced liveness message. Production of liveness
465 // events to the channel is rate-limited by livenessChannelInterval.
466 if sc.liveness != nil {
467 if sc.alive != alive {
468 logger.Info("update-liveness-channel-because-change")
469 sc.liveness <- alive
470 sc.lastLivenessTime = time.Now()
471 } else if time.Since(sc.lastLivenessTime) > sc.livenessChannelInterval {
472 logger.Info("update-liveness-channel-because-interval")
473 sc.liveness <- alive
474 sc.lastLivenessTime = time.Now()
475 }
476 }
477
478 // Only emit a log message when the state changes
479 if sc.alive != alive {
480 logger.Info("set-client-alive", log.Fields{"alive": alive})
481 sc.alive = alive
482 }
483}
484
485// Once unhealthy, we never go back
486func (sc *SaramaClient) setUnhealthy() {
487 sc.healthy = false
488 if sc.healthiness != nil {
489 logger.Infow("set-client-unhealthy", log.Fields{"healthy": sc.healthy})
490 sc.healthiness <- sc.healthy
491 }
492}
493
494func (sc *SaramaClient) isLivenessError(err error) bool {
495 // Sarama producers and consumers encapsulate the error inside
496 // a ProducerError or ConsumerError struct.
497 if prodError, ok := err.(*sarama.ProducerError); ok {
498 err = prodError.Err
499 } else if consumerError, ok := err.(*sarama.ConsumerError); ok {
500 err = consumerError.Err
501 }
502
503 // Sarama-Cluster will compose the error into a ClusterError struct,
504 // which we can't do a compare by reference. To handle that, we the
505 // best we can do is compare the error strings.
506
507 switch err.Error() {
508 case context.DeadlineExceeded.Error():
509 logger.Info("is-liveness-error-timeout")
510 return true
511 case sarama.ErrOutOfBrokers.Error(): // "Kafka: client has run out of available brokers"
512 logger.Info("is-liveness-error-no-brokers")
513 return true
514 case sarama.ErrShuttingDown.Error(): // "Kafka: message received by producer in process of shutting down"
515 logger.Info("is-liveness-error-shutting-down")
516 return true
517 case sarama.ErrControllerNotAvailable.Error(): // "Kafka: controller is not available"
518 logger.Info("is-liveness-error-not-available")
519 return true
520 case breaker.ErrBreakerOpen.Error(): // "circuit breaker is open"
521 logger.Info("is-liveness-error-circuit-breaker-open")
522 return true
523 }
524
525 if strings.HasSuffix(err.Error(), "connection refused") { // "dial tcp 10.244.1.176:9092: connect: connection refused"
526 logger.Info("is-liveness-error-connection-refused")
527 return true
528 }
529
530 if strings.HasSuffix(err.Error(), "i/o timeout") { // "dial tcp 10.244.1.176:9092: i/o timeout"
531 logger.Info("is-liveness-error-io-timeout")
532 return true
533 }
534
535 // Other errors shouldn't trigger a loss of liveness
536
537 logger.Infow("is-liveness-error-ignored", log.Fields{"err": err})
538
539 return false
540}
541
542// send formats and sends the request onto the kafka messaging bus.
543func (sc *SaramaClient) Send(msg interface{}, topic *Topic, keys ...string) error {
544
545 // Assert message is a proto message
546 var protoMsg proto.Message
547 var ok bool
548 // ascertain the value interface type is a proto.Message
549 if protoMsg, ok = msg.(proto.Message); !ok {
550 logger.Warnw("message-not-proto-message", log.Fields{"msg": msg})
551 return fmt.Errorf("not-a-proto-msg-%s", msg)
552 }
553
554 var marshalled []byte
555 var err error
556 // Create the Sarama producer message
557 if marshalled, err = proto.Marshal(protoMsg); err != nil {
558 logger.Errorw("marshalling-failed", log.Fields{"msg": protoMsg, "error": err})
559 return err
560 }
561 key := ""
562 if len(keys) > 0 {
563 key = keys[0] // Only the first key is relevant
564 }
565 kafkaMsg := &sarama.ProducerMessage{
566 Topic: topic.Name,
567 Key: sarama.StringEncoder(key),
568 Value: sarama.ByteEncoder(marshalled),
569 }
570
571 // Send message to kafka
572 sc.producer.Input() <- kafkaMsg
573 // Wait for result
574 // TODO: Use a lock or a different mechanism to ensure the response received corresponds to the message sent.
575 select {
576 case ok := <-sc.producer.Successes():
577 logger.Debugw("message-sent", log.Fields{"status": ok.Topic})
578 sc.updateLiveness(true)
579 case notOk := <-sc.producer.Errors():
580 logger.Debugw("error-sending", log.Fields{"status": notOk})
581 if sc.isLivenessError(notOk) {
582 sc.updateLiveness(false)
583 }
584 return notOk
585 }
586 return nil
587}
588
589// Enable the liveness monitor channel. This channel will report
590// a "true" or "false" on every publish, which indicates whether
591// or not the channel is still live. This channel is then picked up
592// by the service (i.e. rw_core / ro_core) to update readiness status
593// and/or take other actions.
594func (sc *SaramaClient) EnableLivenessChannel(enable bool) chan bool {
595 logger.Infow("kafka-enable-liveness-channel", log.Fields{"enable": enable})
596 if enable {
597 if sc.liveness == nil {
598 logger.Info("kafka-create-liveness-channel")
599 // At least 1, so we can immediately post to it without blocking
600 // Setting a bigger number (10) allows the monitor to fall behind
601 // without blocking others. The monitor shouldn't really fall
602 // behind...
603 sc.liveness = make(chan bool, 10)
604 // post intial state to the channel
605 sc.liveness <- sc.alive
606 }
607 } else {
608 // TODO: Think about whether we need the ability to turn off
609 // liveness monitoring
610 panic("Turning off liveness reporting is not supported")
611 }
612 return sc.liveness
613}
614
615// Enable the Healthiness monitor channel. This channel will report "false"
616// if the kafka consumers die, or some other problem occurs which is
617// catastrophic that would require re-creating the client.
618func (sc *SaramaClient) EnableHealthinessChannel(enable bool) chan bool {
619 logger.Infow("kafka-enable-healthiness-channel", log.Fields{"enable": enable})
620 if enable {
621 if sc.healthiness == nil {
622 logger.Info("kafka-create-healthiness-channel")
623 // At least 1, so we can immediately post to it without blocking
624 // Setting a bigger number (10) allows the monitor to fall behind
625 // without blocking others. The monitor shouldn't really fall
626 // behind...
627 sc.healthiness = make(chan bool, 10)
628 // post intial state to the channel
629 sc.healthiness <- sc.healthy
630 }
631 } else {
632 // TODO: Think about whether we need the ability to turn off
633 // liveness monitoring
634 panic("Turning off healthiness reporting is not supported")
635 }
636 return sc.healthiness
637}
638
639// send an empty message on the liveness channel to check whether connectivity has
640// been restored.
641func (sc *SaramaClient) SendLiveness() error {
642 if !sc.started {
643 return fmt.Errorf("SendLiveness() called while not started")
644 }
645
646 kafkaMsg := &sarama.ProducerMessage{
647 Topic: "_liveness_test",
648 Value: sarama.StringEncoder(time.Now().Format(time.RFC3339)), // for debugging / informative use
649 }
650
651 // Send message to kafka
652 sc.producer.Input() <- kafkaMsg
653 // Wait for result
654 // TODO: Use a lock or a different mechanism to ensure the response received corresponds to the message sent.
655 select {
656 case ok := <-sc.producer.Successes():
657 logger.Debugw("liveness-message-sent", log.Fields{"status": ok.Topic})
658 sc.updateLiveness(true)
659 case notOk := <-sc.producer.Errors():
660 logger.Debugw("liveness-error-sending", log.Fields{"status": notOk})
661 if sc.isLivenessError(notOk) {
662 sc.updateLiveness(false)
663 }
664 return notOk
665 }
666 return nil
667}
668
669// getGroupId returns the group id from the key-value args.
670func getGroupId(kvArgs ...*KVArg) string {
671 for _, arg := range kvArgs {
672 if arg.Key == GroupIdKey {
673 return arg.Value.(string)
674 }
675 }
676 return ""
677}
678
679// getOffset returns the offset from the key-value args.
680func getOffset(kvArgs ...*KVArg) int64 {
681 for _, arg := range kvArgs {
682 if arg.Key == Offset {
683 return arg.Value.(int64)
684 }
685 }
686 return sarama.OffsetNewest
687}
688
689func (sc *SaramaClient) createClusterAdmin() error {
690 config := sarama.NewConfig()
691 config.Version = sarama.V1_0_0_0
692
693 // Create a cluster Admin
694 var cAdmin sarama.ClusterAdmin
695 var err error
696 if cAdmin, err = sarama.NewClusterAdmin([]string{sc.KafkaAddress}, config); err != nil {
697 logger.Errorw("cluster-admin-failure", log.Fields{"error": err, "broker-address": sc.KafkaAddress})
698 return err
699 }
700 sc.cAdmin = cAdmin
701 return nil
702}
703
704func (sc *SaramaClient) lockTopic(topic *Topic) {
705 sc.lockOfTopicLockMap.Lock()
706 if _, exist := sc.topicLockMap[topic.Name]; exist {
707 sc.lockOfTopicLockMap.Unlock()
708 sc.topicLockMap[topic.Name].Lock()
709 } else {
710 sc.topicLockMap[topic.Name] = &sync.RWMutex{}
711 sc.lockOfTopicLockMap.Unlock()
712 sc.topicLockMap[topic.Name].Lock()
713 }
714}
715
716func (sc *SaramaClient) unLockTopic(topic *Topic) {
717 sc.lockOfTopicLockMap.Lock()
718 defer sc.lockOfTopicLockMap.Unlock()
719 if _, exist := sc.topicLockMap[topic.Name]; exist {
720 sc.topicLockMap[topic.Name].Unlock()
721 }
722}
723
724func (sc *SaramaClient) addTopicToConsumerChannelMap(id string, arg *consumerChannels) {
725 sc.lockTopicToConsumerChannelMap.Lock()
726 defer sc.lockTopicToConsumerChannelMap.Unlock()
727 if _, exist := sc.topicToConsumerChannelMap[id]; !exist {
728 sc.topicToConsumerChannelMap[id] = arg
729 }
730}
731
732func (sc *SaramaClient) getConsumerChannel(topic *Topic) *consumerChannels {
733 sc.lockTopicToConsumerChannelMap.RLock()
734 defer sc.lockTopicToConsumerChannelMap.RUnlock()
735
736 if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
737 return consumerCh
738 }
739 return nil
740}
741
742func (sc *SaramaClient) addChannelToConsumerChannelMap(topic *Topic, ch chan *ic.InterContainerMessage) {
743 sc.lockTopicToConsumerChannelMap.Lock()
744 defer sc.lockTopicToConsumerChannelMap.Unlock()
745 if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
746 consumerCh.channels = append(consumerCh.channels, ch)
747 return
748 }
749 logger.Warnw("consumers-channel-not-exist", log.Fields{"topic": topic.Name})
750}
751
752//closeConsumers closes a list of sarama consumers. The consumers can either be a partition consumers or a group consumers
753func closeConsumers(consumers []interface{}) error {
754 var err error
755 for _, consumer := range consumers {
756 // Is it a partition consumers?
757 if partionConsumer, ok := consumer.(sarama.PartitionConsumer); ok {
758 if errTemp := partionConsumer.Close(); errTemp != nil {
759 logger.Debugw("partition!!!", log.Fields{"err": errTemp})
760 if strings.Compare(errTemp.Error(), sarama.ErrUnknownTopicOrPartition.Error()) == 0 {
761 // This can occur on race condition
762 err = nil
763 } else {
764 err = errTemp
765 }
766 }
767 } else if groupConsumer, ok := consumer.(*scc.Consumer); ok {
768 if errTemp := groupConsumer.Close(); errTemp != nil {
769 if strings.Compare(errTemp.Error(), sarama.ErrUnknownTopicOrPartition.Error()) == 0 {
770 // This can occur on race condition
771 err = nil
772 } else {
773 err = errTemp
774 }
775 }
776 }
777 }
778 return err
779}
780
781func (sc *SaramaClient) removeChannelFromConsumerChannelMap(topic Topic, ch <-chan *ic.InterContainerMessage) error {
782 sc.lockTopicToConsumerChannelMap.Lock()
783 defer sc.lockTopicToConsumerChannelMap.Unlock()
784 if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
785 // Channel will be closed in the removeChannel method
786 consumerCh.channels = removeChannel(consumerCh.channels, ch)
787 // If there are no more channels then we can close the consumers itself
788 if len(consumerCh.channels) == 0 {
789 logger.Debugw("closing-consumers", log.Fields{"topic": topic})
790 err := closeConsumers(consumerCh.consumers)
791 //err := consumerCh.consumers.Close()
792 delete(sc.topicToConsumerChannelMap, topic.Name)
793 return err
794 }
795 return nil
796 }
797 logger.Warnw("topic-does-not-exist", log.Fields{"topic": topic.Name})
798 return errors.New("topic-does-not-exist")
799}
800
801func (sc *SaramaClient) clearTopicFromConsumerChannelMap(topic Topic) error {
802 sc.lockTopicToConsumerChannelMap.Lock()
803 defer sc.lockTopicToConsumerChannelMap.Unlock()
804 if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
805 for _, ch := range consumerCh.channels {
806 // Channel will be closed in the removeChannel method
807 removeChannel(consumerCh.channels, ch)
808 }
809 err := closeConsumers(consumerCh.consumers)
810 //if err == sarama.ErrUnknownTopicOrPartition {
811 // // Not an error
812 // err = nil
813 //}
814 //err := consumerCh.consumers.Close()
815 delete(sc.topicToConsumerChannelMap, topic.Name)
816 return err
817 }
818 logger.Debugw("topic-does-not-exist", log.Fields{"topic": topic.Name})
819 return nil
820}
821
822//createPublisher creates the publisher which is used to send a message onto kafka
823func (sc *SaramaClient) createPublisher() error {
824 // This Creates the publisher
825 config := sarama.NewConfig()
826 config.Producer.Partitioner = sarama.NewRandomPartitioner
827 config.Producer.Flush.Frequency = time.Duration(sc.producerFlushFrequency)
828 config.Producer.Flush.Messages = sc.producerFlushMessages
829 config.Producer.Flush.MaxMessages = sc.producerFlushMaxmessages
830 config.Producer.Return.Errors = sc.producerReturnErrors
831 config.Producer.Return.Successes = sc.producerReturnSuccess
832 //config.Producer.RequiredAcks = sarama.WaitForAll
833 config.Producer.RequiredAcks = sarama.WaitForLocal
834
835 brokers := []string{sc.KafkaAddress}
836
837 if producer, err := sarama.NewAsyncProducer(brokers, config); err != nil {
838 logger.Errorw("error-starting-publisher", log.Fields{"error": err})
839 return err
840 } else {
841 sc.producer = producer
842 }
843 logger.Info("Kafka-publisher-created")
844 return nil
845}
846
847func (sc *SaramaClient) createConsumer() error {
848 config := sarama.NewConfig()
849 config.Consumer.Return.Errors = true
850 config.Consumer.Fetch.Min = 1
851 config.Consumer.MaxWaitTime = time.Duration(sc.consumerMaxwait) * time.Millisecond
852 config.Consumer.MaxProcessingTime = time.Duration(sc.maxProcessingTime) * time.Millisecond
853 config.Consumer.Offsets.Initial = sarama.OffsetNewest
854 config.Metadata.Retry.Max = sc.metadataMaxRetry
855 brokers := []string{sc.KafkaAddress}
856
857 if consumer, err := sarama.NewConsumer(brokers, config); err != nil {
858 logger.Errorw("error-starting-consumers", log.Fields{"error": err})
859 return err
860 } else {
861 sc.consumer = consumer
862 }
863 logger.Info("Kafka-consumers-created")
864 return nil
865}
866
867// createGroupConsumer creates a consumers group
868func (sc *SaramaClient) createGroupConsumer(topic *Topic, groupId string, initialOffset int64, retries int) (*scc.Consumer, error) {
869 config := scc.NewConfig()
870 config.ClientID = uuid.New().String()
871 config.Group.Mode = scc.ConsumerModeMultiplex
872 config.Consumer.Group.Heartbeat.Interval, _ = time.ParseDuration("1s")
873 config.Consumer.Return.Errors = true
874 //config.Group.Return.Notifications = false
875 //config.Consumer.MaxWaitTime = time.Duration(DefaultConsumerMaxwait) * time.Millisecond
876 //config.Consumer.MaxProcessingTime = time.Duration(DefaultMaxProcessingTime) * time.Millisecond
877 config.Consumer.Offsets.Initial = initialOffset
878 //config.Consumer.Offsets.Initial = sarama.OffsetOldest
879 brokers := []string{sc.KafkaAddress}
880
881 topics := []string{topic.Name}
882 var consumer *scc.Consumer
883 var err error
884
885 if consumer, err = scc.NewConsumer(brokers, groupId, topics, config); err != nil {
886 logger.Errorw("create-group-consumers-failure", log.Fields{"error": err, "topic": topic.Name, "groupId": groupId})
887 return nil, err
888 }
889 logger.Debugw("create-group-consumers-success", log.Fields{"topic": topic.Name, "groupId": groupId})
890
891 //sc.groupConsumers[topic.Name] = consumer
892 sc.addToGroupConsumers(topic.Name, consumer)
893 return consumer, nil
894}
895
896// dispatchToConsumers sends the intercontainermessage received on a given topic to all subscribers for that
897// topic via the unique channel each subscriber received during subscription
898func (sc *SaramaClient) dispatchToConsumers(consumerCh *consumerChannels, protoMessage *ic.InterContainerMessage) {
899 // Need to go over all channels and publish messages to them - do we need to copy msg?
900 sc.lockTopicToConsumerChannelMap.RLock()
901 for _, ch := range consumerCh.channels {
902 go func(c chan *ic.InterContainerMessage) {
903 c <- protoMessage
904 }(ch)
905 }
906 sc.lockTopicToConsumerChannelMap.RUnlock()
907
908 if callback := sc.metadataCallback; callback != nil {
909 ts, _ := ptypes.Timestamp(protoMessage.Header.Timestamp)
910 callback(protoMessage.Header.FromTopic, ts)
911 }
912}
913
914func (sc *SaramaClient) consumeFromAPartition(topic *Topic, consumer sarama.PartitionConsumer, consumerChnls *consumerChannels) {
915 logger.Debugw("starting-partition-consumption-loop", log.Fields{"topic": topic.Name})
916startloop:
917 for {
918 select {
919 case err, ok := <-consumer.Errors():
920 if ok {
921 if sc.isLivenessError(err) {
922 sc.updateLiveness(false)
923 logger.Warnw("partition-consumers-error", log.Fields{"error": err})
924 }
925 } else {
926 // Channel is closed
927 break startloop
928 }
929 case msg, ok := <-consumer.Messages():
930 //logger.Debugw("message-received", logger.Fields{"msg": msg, "receivedTopic": msg.Topic})
931 if !ok {
932 // channel is closed
933 break startloop
934 }
935 msgBody := msg.Value
936 sc.updateLiveness(true)
937 logger.Debugw("message-received", log.Fields{"timestamp": msg.Timestamp, "receivedTopic": msg.Topic})
938 icm := &ic.InterContainerMessage{}
939 if err := proto.Unmarshal(msgBody, icm); err != nil {
940 logger.Warnw("partition-invalid-message", log.Fields{"error": err})
941 continue
942 }
943 go sc.dispatchToConsumers(consumerChnls, icm)
944 case <-sc.doneCh:
945 logger.Infow("partition-received-exit-signal", log.Fields{"topic": topic.Name})
946 break startloop
947 }
948 }
949 logger.Infow("partition-consumer-stopped", log.Fields{"topic": topic.Name})
950 sc.setUnhealthy()
951}
952
953func (sc *SaramaClient) consumeGroupMessages(topic *Topic, consumer *scc.Consumer, consumerChnls *consumerChannels) {
954 logger.Debugw("starting-group-consumption-loop", log.Fields{"topic": topic.Name})
955
956startloop:
957 for {
958 select {
959 case err, ok := <-consumer.Errors():
960 if ok {
961 if sc.isLivenessError(err) {
962 sc.updateLiveness(false)
963 }
964 logger.Warnw("group-consumers-error", log.Fields{"topic": topic.Name, "error": err})
965 } else {
966 logger.Warnw("group-consumers-closed-err", log.Fields{"topic": topic.Name})
967 // channel is closed
968 break startloop
969 }
970 case msg, ok := <-consumer.Messages():
971 if !ok {
972 logger.Warnw("group-consumers-closed-msg", log.Fields{"topic": topic.Name})
973 // Channel closed
974 break startloop
975 }
976 sc.updateLiveness(true)
977 logger.Debugw("message-received", log.Fields{"timestamp": msg.Timestamp, "receivedTopic": msg.Topic})
978 msgBody := msg.Value
979 icm := &ic.InterContainerMessage{}
980 if err := proto.Unmarshal(msgBody, icm); err != nil {
981 logger.Warnw("invalid-message", log.Fields{"error": err})
982 continue
983 }
984 go sc.dispatchToConsumers(consumerChnls, icm)
985 consumer.MarkOffset(msg, "")
986 case ntf := <-consumer.Notifications():
987 logger.Debugw("group-received-notification", log.Fields{"notification": ntf})
988 case <-sc.doneCh:
989 logger.Infow("group-received-exit-signal", log.Fields{"topic": topic.Name})
990 break startloop
991 }
992 }
993 logger.Infow("group-consumer-stopped", log.Fields{"topic": topic.Name})
994 sc.setUnhealthy()
995}
996
997func (sc *SaramaClient) startConsumers(topic *Topic) error {
998 logger.Debugw("starting-consumers", log.Fields{"topic": topic.Name})
999 var consumerCh *consumerChannels
1000 if consumerCh = sc.getConsumerChannel(topic); consumerCh == nil {
1001 logger.Errorw("consumers-not-exist", log.Fields{"topic": topic.Name})
1002 return errors.New("consumers-not-exist")
1003 }
1004 // For each consumer listening for that topic, start a consumption loop
1005 for _, consumer := range consumerCh.consumers {
1006 if pConsumer, ok := consumer.(sarama.PartitionConsumer); ok {
1007 go sc.consumeFromAPartition(topic, pConsumer, consumerCh)
1008 } else if gConsumer, ok := consumer.(*scc.Consumer); ok {
1009 go sc.consumeGroupMessages(topic, gConsumer, consumerCh)
1010 } else {
1011 logger.Errorw("invalid-consumer", log.Fields{"topic": topic})
1012 return errors.New("invalid-consumer")
1013 }
1014 }
1015 return nil
1016}
1017
1018//// setupConsumerChannel creates a consumerChannels object for that topic and add it to the consumerChannels map
1019//// for that topic. It also starts the routine that listens for messages on that topic.
1020func (sc *SaramaClient) setupPartitionConsumerChannel(topic *Topic, initialOffset int64) (chan *ic.InterContainerMessage, error) {
1021 var pConsumers []sarama.PartitionConsumer
1022 var err error
1023
1024 if pConsumers, err = sc.createPartitionConsumers(topic, initialOffset); err != nil {
1025 logger.Errorw("creating-partition-consumers-failure", log.Fields{"error": err, "topic": topic.Name})
1026 return nil, err
1027 }
1028
1029 consumersIf := make([]interface{}, 0)
1030 for _, pConsumer := range pConsumers {
1031 consumersIf = append(consumersIf, pConsumer)
1032 }
1033
1034 // Create the consumers/channel structure and set the consumers and create a channel on that topic - for now
1035 // unbuffered to verify race conditions.
1036 consumerListeningChannel := make(chan *ic.InterContainerMessage)
1037 cc := &consumerChannels{
1038 consumers: consumersIf,
1039 channels: []chan *ic.InterContainerMessage{consumerListeningChannel},
1040 }
1041
1042 // Add the consumers channel to the map
1043 sc.addTopicToConsumerChannelMap(topic.Name, cc)
1044
1045 //Start a consumers to listen on that specific topic
1046 go func() {
1047 if err := sc.startConsumers(topic); err != nil {
1048 logger.Errorw("start-consumers-failed", log.Fields{
1049 "topic": topic,
1050 "error": err})
1051 }
1052 }()
1053
1054 return consumerListeningChannel, nil
1055}
1056
1057// setupConsumerChannel creates a consumerChannels object for that topic and add it to the consumerChannels map
1058// for that topic. It also starts the routine that listens for messages on that topic.
1059func (sc *SaramaClient) setupGroupConsumerChannel(topic *Topic, groupId string, initialOffset int64) (chan *ic.InterContainerMessage, error) {
1060 // TODO: Replace this development partition consumers with a group consumers
1061 var pConsumer *scc.Consumer
1062 var err error
1063 if pConsumer, err = sc.createGroupConsumer(topic, groupId, initialOffset, DefaultMaxRetries); err != nil {
1064 logger.Errorw("creating-partition-consumers-failure", log.Fields{"error": err, "topic": topic.Name})
1065 return nil, err
1066 }
1067 // Create the consumers/channel structure and set the consumers and create a channel on that topic - for now
1068 // unbuffered to verify race conditions.
1069 consumerListeningChannel := make(chan *ic.InterContainerMessage)
1070 cc := &consumerChannels{
1071 consumers: []interface{}{pConsumer},
1072 channels: []chan *ic.InterContainerMessage{consumerListeningChannel},
1073 }
1074
1075 // Add the consumers channel to the map
1076 sc.addTopicToConsumerChannelMap(topic.Name, cc)
1077
1078 //Start a consumers to listen on that specific topic
1079 go func() {
1080 if err := sc.startConsumers(topic); err != nil {
1081 logger.Errorw("start-consumers-failed", log.Fields{
1082 "topic": topic,
1083 "error": err})
1084 }
1085 }()
1086
1087 return consumerListeningChannel, nil
1088}
1089
1090func (sc *SaramaClient) createPartitionConsumers(topic *Topic, initialOffset int64) ([]sarama.PartitionConsumer, error) {
1091 logger.Debugw("creating-partition-consumers", log.Fields{"topic": topic.Name})
1092 partitionList, err := sc.consumer.Partitions(topic.Name)
1093 if err != nil {
1094 logger.Warnw("get-partition-failure", log.Fields{"error": err, "topic": topic.Name})
1095 return nil, err
1096 }
1097
1098 pConsumers := make([]sarama.PartitionConsumer, 0)
1099 for _, partition := range partitionList {
1100 var pConsumer sarama.PartitionConsumer
1101 if pConsumer, err = sc.consumer.ConsumePartition(topic.Name, partition, initialOffset); err != nil {
1102 logger.Warnw("consumers-partition-failure", log.Fields{"error": err, "topic": topic.Name})
1103 return nil, err
1104 }
1105 pConsumers = append(pConsumers, pConsumer)
1106 }
1107 return pConsumers, nil
1108}
1109
1110func removeChannel(channels []chan *ic.InterContainerMessage, ch <-chan *ic.InterContainerMessage) []chan *ic.InterContainerMessage {
1111 var i int
1112 var channel chan *ic.InterContainerMessage
1113 for i, channel = range channels {
1114 if channel == ch {
1115 channels[len(channels)-1], channels[i] = channels[i], channels[len(channels)-1]
1116 close(channel)
1117 logger.Debug("channel-closed")
1118 return channels[:len(channels)-1]
1119 }
1120 }
1121 return channels
1122}
1123
1124func (sc *SaramaClient) addToGroupConsumers(topic string, consumer *scc.Consumer) {
1125 sc.lockOfGroupConsumers.Lock()
1126 defer sc.lockOfGroupConsumers.Unlock()
1127 if _, exist := sc.groupConsumers[topic]; !exist {
1128 sc.groupConsumers[topic] = consumer
1129 }
1130}
1131
1132func (sc *SaramaClient) deleteFromGroupConsumers(topic string) error {
1133 sc.lockOfGroupConsumers.Lock()
1134 defer sc.lockOfGroupConsumers.Unlock()
1135 if _, exist := sc.groupConsumers[topic]; exist {
1136 consumer := sc.groupConsumers[topic]
1137 delete(sc.groupConsumers, topic)
1138 if err := consumer.Close(); err != nil {
1139 logger.Errorw("failure-closing-consumer", log.Fields{"error": err})
1140 return err
1141 }
1142 }
1143 return nil
1144}