github.com/rohankumardubey/aresdb@v0.0.2-0.20190517170215-e54e3ca06b9c/subscriber/common/consumer/kafka/kafka_confluent.go (about) 1 // Copyright (c) 2017-2018 Uber Technologies, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package kafka 16 17 import ( 18 "fmt" 19 "github.com/uber/aresdb/subscriber/common/consumer" 20 "sync" 21 22 kafkaConfluent "github.com/confluentinc/confluent-kafka-go/kafka" 23 "github.com/uber-go/tally" 24 "github.com/uber/aresdb/subscriber/common/rules" 25 "github.com/uber/aresdb/subscriber/config" 26 "github.com/uber/aresdb/utils" 27 "go.uber.org/zap" 28 "strconv" 29 ) 30 31 // KafkaConsumer implements Consumer interface 32 type KafkaConsumer struct { 33 *kafkaConfluent.Consumer 34 kafkaConfluent.ConfigMap 35 sync.Mutex 36 37 TopicArray []string 38 Logger *zap.Logger 39 Scope tally.Scope 40 ErrCh chan error 41 MsgCh chan consumer.Message 42 43 // WARNING: The following channels should not be closed by the lib users 44 CloseAttempted bool 45 CloseErr error 46 CloseCh chan struct{} 47 } 48 49 // KafkaMessage implements Message interface 50 type KafkaMessage struct { 51 *kafkaConfluent.Message 52 53 Consumer consumer.Consumer 54 ClusterName string 55 } 56 57 // NewKafkaConsumer creates kafka consumer by using https://github.com/confluentinc/confluent-kafka-go. 58 func NewKafkaConsumer(jobConfig *rules.JobConfig, serviceConfig config.ServiceConfig) (consumer.Consumer, error) { 59 offsetReset := "earliest" 60 if jobConfig.StreamingConfig.LatestOffset { 61 offsetReset = "latest" 62 } 63 cfg := kafkaConfluent.ConfigMap{ 64 "bootstrap.servers": jobConfig.StreamingConfig.KafkaBroker, 65 "group.id": GetConsumerGroupName(serviceConfig.Environment.Deployment, jobConfig.Name, jobConfig.AresTableConfig.Cluster), 66 "max.poll.interval.ms": jobConfig.StreamingConfig.MaxPollIntervalMs, 67 "session.timeout.ms": jobConfig.StreamingConfig.SessionTimeoutNs, 68 "go.events.channel.enable": false, 69 "go.application.rebalance.enable": false, 70 "enable.partition.eof": true, 71 "auto.offset.reset": offsetReset, 72 } 73 serviceConfig.Logger.Info("Kafka consumer", 74 zap.String("job", jobConfig.Name), 75 zap.String("broker", jobConfig.StreamingConfig.KafkaBroker), 76 zap.Any("config", cfg)) 77 78 c, err := kafkaConfluent.NewConsumer(&cfg) 79 if err != nil { 80 return nil, utils.StackError(err, "Unable to initialize Kafka consumer") 81 } 82 83 err = c.Subscribe(jobConfig.StreamingConfig.Topic, nil) 84 if err != nil { 85 return nil, utils.StackError(err, fmt.Sprintf("Unable to subscribe to topic: %s", jobConfig.StreamingConfig.Topic)) 86 } 87 88 logger := serviceConfig.Logger.With( 89 zap.String("kafkaBroker", jobConfig.StreamingConfig.KafkaBroker), 90 zap.String("topic", jobConfig.StreamingConfig.Topic), 91 ) 92 93 scope := serviceConfig.Scope.Tagged(map[string]string{ 94 "broker": jobConfig.StreamingConfig.KafkaBroker, 95 }) 96 97 kc := KafkaConsumer{ 98 Consumer: c, 99 ConfigMap: cfg, 100 TopicArray: []string{jobConfig.StreamingConfig.Topic}, 101 Logger: logger, 102 Scope: scope, 103 ErrCh: make(chan error, jobConfig.StreamingConfig.ChannelBufferSize), 104 MsgCh: make(chan consumer.Message, jobConfig.StreamingConfig.ChannelBufferSize), 105 CloseCh: make(chan struct{}), 106 } 107 108 go kc.startConsuming() 109 return &kc, nil 110 } 111 112 // Name returns the name of this consumer group. 113 func (c *KafkaConsumer) Name() string { 114 return c.ConfigMap["group.id"].(string) 115 } 116 117 // Topics returns the names of the topics being consumed. 118 func (c *KafkaConsumer) Topics() []string { 119 return append([]string(nil), c.TopicArray...) 120 } 121 122 // Errors returns a channel of errors for the topic. To prevent deadlocks, 123 // users must read from the error channel. 124 // 125 // All errors returned from this channel can be safely cast to the 126 // consumer.Error interface, which allows structured access to the topic 127 // name and partition number. 128 func (c *KafkaConsumer) Errors() <-chan error { 129 return c.ErrCh 130 } 131 132 // Closed returns a channel that unblocks when the consumer successfully shuts 133 // down. 134 func (c *KafkaConsumer) Closed() <-chan struct{} { 135 return c.CloseCh 136 } 137 138 // Messages returns a channel of messages for the topic. 139 // 140 // If the consumer is not configured with nonzero buffer size, the Errors() 141 // channel must be read in conjunction with Messages() to prevent deadlocks. 142 func (c *KafkaConsumer) Messages() <-chan consumer.Message { 143 return c.MsgCh 144 } 145 146 // CommitUpTo marks this message and all previous messages in the same partition 147 // as processed. The last processed offset for each partition is periodically 148 // flushed to ZooKeeper; on startup, consumers begin processing after the last 149 // stored offset. 150 func (c *KafkaConsumer) CommitUpTo(msg consumer.Message) error { 151 if concreteMsg, ok := msg.(*KafkaMessage); ok { 152 // Just unwrap the underlying message. 153 c.CommitMessage(concreteMsg.Message) 154 } else { 155 topic := msg.Topic() 156 c.CommitOffsets([]kafkaConfluent.TopicPartition{ 157 {&topic, 158 msg.Partition(), 159 kafkaConfluent.Offset(msg.Offset()), 160 nil, 161 }, 162 }) 163 } 164 return nil 165 } 166 167 func (c *KafkaConsumer) startConsuming() { 168 c.Logger.Debug("Start consumption goroutine") 169 170 // those four Metrics are of the format {"<topic name>":{<partition id>: <offset>, ...}, ...} 171 msgCounter := make(map[string]map[int32]tally.Counter) 172 msgByteCounter := make(map[string]map[int32]tally.Counter) 173 msgOffsetGauge := make(map[string]map[int32]tally.Gauge) 174 msgLagGauge := make(map[string]map[int32]tally.Gauge) 175 176 // initialize outer map 177 for _, topic := range c.TopicArray { 178 msgCounter[topic] = make(map[int32]tally.Counter) 179 msgByteCounter[topic] = make(map[int32]tally.Counter) 180 msgOffsetGauge[topic] = make(map[int32]tally.Gauge) 181 msgLagGauge[topic] = make(map[int32]tally.Gauge) 182 } 183 184 for run := true; run; { 185 select { 186 case _ = <-c.CloseCh: 187 c.Logger.Info("Received close Signal") 188 run = false 189 case event := <-c.Events(): 190 switch e := event.(type) { 191 case *kafkaConfluent.Message: 192 c.processMsg(e, msgCounter, msgByteCounter, msgOffsetGauge, msgLagGauge) 193 case kafkaConfluent.Error: 194 c.ErrCh <- e 195 c.Logger.Error("Received error event", zap.Error(e)) 196 default: 197 c.Logger.Info("Ignored consumer event", zap.Any("event", e)) 198 } 199 } 200 } 201 } 202 203 func (c *KafkaConsumer) processMsg(msg *kafkaConfluent.Message, 204 msgCounter map[string]map[int32]tally.Counter, 205 msgByteCounter map[string]map[int32]tally.Counter, 206 msgOffsetGauge map[string]map[int32]tally.Gauge, 207 msgLagGauge map[string]map[int32]tally.Gauge) { 208 209 c.Logger.Debug("Received nessage event", zap.Any("message", msg)) 210 c.MsgCh <- &KafkaMessage{ 211 Message: msg, 212 Consumer: c, 213 } 214 215 topic := *msg.TopicPartition.Topic 216 partition := msg.TopicPartition.Partition 217 pncm := msgCounter[topic] 218 nCounter, ok := pncm[partition] 219 if !ok { 220 nCounter = c.Scope.Tagged(map[string]string{"topic": topic, "partition": strconv.Itoa(int(partition))}).Counter("messages-count") 221 pncm[partition] = nCounter 222 } 223 nCounter.Inc(1) 224 225 pbcm := msgByteCounter[topic] 226 bCounter, ok := pbcm[partition] 227 if !ok { 228 bCounter = c.Scope.Tagged(map[string]string{"topic": topic, "partition": strconv.Itoa(int(partition))}).Counter("message-bytes-count") 229 pbcm[partition] = bCounter 230 } 231 bCounter.Inc(int64(len(msg.Value))) 232 233 pogm := msgOffsetGauge[topic] 234 oGauge, ok := pogm[partition] 235 if !ok { 236 oGauge = c.Scope.Tagged(map[string]string{"topic": topic, "partition": strconv.Itoa(int(partition))}).Gauge("latest-offset") 237 pogm[partition] = oGauge 238 } 239 oGauge.Update(float64(msg.TopicPartition.Offset)) 240 241 plgm := msgLagGauge[topic] 242 lGauge, ok := plgm[partition] 243 if !ok { 244 lGauge = c.Scope.Tagged(map[string]string{"topic": topic, "partition": strconv.Itoa(int(partition))}).Gauge("offset-lag") 245 } 246 247 _, offset, _ := c.Consumer.QueryWatermarkOffsets(topic, partition, 100) 248 249 if offset > int64(msg.TopicPartition.Offset) { 250 lGauge.Update(float64(offset - int64(msg.TopicPartition.Offset) - 1)) 251 } else { 252 lGauge.Update(0) 253 } 254 } 255 256 func (c *KafkaConsumer) Close() error { 257 c.Lock() 258 defer c.Unlock() 259 260 if c.CloseAttempted { 261 return fmt.Errorf("Close attempted again on consumer group %s", c.ConfigMap["group.id"].(string)) 262 } 263 264 c.Logger.Debug("Attempting to close consumer", 265 zap.String("consumerGroup", c.ConfigMap["group.id"].(string))) 266 c.CloseErr = c.Consumer.Close() 267 if c.CloseErr != nil { 268 c.Logger.With(zap.NamedError("error", c.CloseErr)).Error("Failed to close consumer", 269 zap.String("consumerGroup", c.ConfigMap["group.id"].(string))) 270 } else { 271 c.Logger.Debug("Started to close consumer", 272 zap.String("consumerGroup", c.ConfigMap["group.id"].(string))) 273 } 274 close(c.CloseCh) 275 c.CloseAttempted = true 276 return c.CloseErr 277 } 278 279 func (m *KafkaMessage) Key() []byte { 280 return m.Message.Key 281 } 282 283 func (m *KafkaMessage) Value() []byte { 284 return m.Message.Value 285 } 286 287 func (m *KafkaMessage) Topic() string { 288 return *m.TopicPartition.Topic 289 } 290 291 func (m *KafkaMessage) Partition() int32 { 292 return m.TopicPartition.Partition 293 } 294 295 func (m *KafkaMessage) Offset() int64 { 296 return int64(m.TopicPartition.Offset) 297 } 298 299 func (m *KafkaMessage) Ack() { 300 if m.Consumer != nil { 301 m.Consumer.CommitUpTo(m) 302 } 303 } 304 305 func (m *KafkaMessage) Nack() { 306 // No op for now since Kafka based DLQ is not implemented 307 } 308 309 func (m *KafkaMessage) Cluster() string { 310 return m.ClusterName 311 } 312 313 // GetConsumerGroupName will return the consumer group name to use or being used 314 // for given deployment and job name 315 func GetConsumerGroupName(deployment, jobName string, aresCluster string) string { 316 return fmt.Sprintf("ares-subscriber_%s_%s_%s_streaming", deployment, jobName, aresCluster) 317 }