github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/kafka_balanced.go (about) 1 package reader 2 3 import ( 4 "context" 5 "crypto/tls" 6 "errors" 7 "fmt" 8 "io" 9 "strconv" 10 "strings" 11 "sync" 12 "time" 13 14 "github.com/Jeffail/benthos/v3/lib/log" 15 "github.com/Jeffail/benthos/v3/lib/message" 16 "github.com/Jeffail/benthos/v3/lib/message/batch" 17 "github.com/Jeffail/benthos/v3/lib/metrics" 18 "github.com/Jeffail/benthos/v3/lib/types" 19 "github.com/Jeffail/benthos/v3/lib/util/kafka/sasl" 20 btls "github.com/Jeffail/benthos/v3/lib/util/tls" 21 "github.com/Shopify/sarama" 22 ) 23 24 //------------------------------------------------------------------------------ 25 26 // KafkaBalancedGroupConfig contains config fields for Kafka consumer groups. 27 type KafkaBalancedGroupConfig struct { 28 SessionTimeout string `json:"session_timeout" yaml:"session_timeout"` 29 HeartbeatInterval string `json:"heartbeat_interval" yaml:"heartbeat_interval"` 30 RebalanceTimeout string `json:"rebalance_timeout" yaml:"rebalance_timeout"` 31 } 32 33 // NewKafkaBalancedGroupConfig returns a KafkaBalancedGroupConfig with default 34 // values. 35 func NewKafkaBalancedGroupConfig() KafkaBalancedGroupConfig { 36 return KafkaBalancedGroupConfig{ 37 SessionTimeout: "10s", 38 HeartbeatInterval: "3s", 39 RebalanceTimeout: "60s", 40 } 41 } 42 43 // KafkaBalancedConfig contains configuration for the KafkaBalanced input type. 44 type KafkaBalancedConfig struct { 45 Addresses []string `json:"addresses" yaml:"addresses"` 46 ClientID string `json:"client_id" yaml:"client_id"` 47 RackID string `json:"rack_id" yaml:"rack_id"` 48 ConsumerGroup string `json:"consumer_group" yaml:"consumer_group"` 49 Group KafkaBalancedGroupConfig `json:"group" yaml:"group"` 50 CommitPeriod string `json:"commit_period" yaml:"commit_period"` 51 MaxProcessingPeriod string `json:"max_processing_period" yaml:"max_processing_period"` 52 FetchBufferCap int `json:"fetch_buffer_cap" yaml:"fetch_buffer_cap"` 53 Topics []string `json:"topics" yaml:"topics"` 54 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 55 StartFromOldest bool `json:"start_from_oldest" yaml:"start_from_oldest"` 56 TargetVersion string `json:"target_version" yaml:"target_version"` 57 // TODO: V4 Remove this. 58 MaxBatchCount int `json:"max_batch_count" yaml:"max_batch_count"` 59 TLS btls.Config `json:"tls" yaml:"tls"` 60 SASL sasl.Config `json:"sasl" yaml:"sasl"` 61 } 62 63 // NewKafkaBalancedConfig creates a new KafkaBalancedConfig with default values. 64 // TODO: V4 Remove this unused implementation. 65 func NewKafkaBalancedConfig() KafkaBalancedConfig { 66 return KafkaBalancedConfig{ 67 Addresses: []string{"localhost:9092"}, 68 ClientID: "benthos_kafka_input", 69 RackID: "", 70 ConsumerGroup: "benthos_consumer_group", 71 Group: NewKafkaBalancedGroupConfig(), 72 CommitPeriod: "1s", 73 MaxProcessingPeriod: "100ms", 74 FetchBufferCap: 256, 75 Topics: []string{"benthos_stream"}, 76 StartFromOldest: true, 77 TargetVersion: sarama.V1_0_0_0.String(), 78 Batching: batch.NewPolicyConfig(), 79 MaxBatchCount: 1, 80 TLS: btls.NewConfig(), 81 SASL: sasl.NewConfig(), 82 } 83 } 84 85 //------------------------------------------------------------------------------ 86 87 type consumerMessage struct { 88 *sarama.ConsumerMessage 89 highWaterMark int64 90 } 91 92 // KafkaBalanced is an input type that reads from a Kafka cluster by balancing 93 // partitions across other consumers of the same consumer group. 94 type KafkaBalanced struct { 95 version sarama.KafkaVersion 96 tlsConf *tls.Config 97 addresses []string 98 topics []string 99 100 commitPeriod time.Duration 101 sessionTimeout time.Duration 102 heartbeatInterval time.Duration 103 rebalanceTimeout time.Duration 104 maxProcPeriod time.Duration 105 106 cMut sync.Mutex 107 groupCancelFn context.CancelFunc 108 session sarama.ConsumerGroupSession 109 msgChan chan consumerMessage 110 111 offsets map[string]map[int32]int64 112 113 mRebalanced metrics.StatCounter 114 115 conf KafkaBalancedConfig 116 stats metrics.Type 117 log log.Modular 118 mgr types.Manager 119 } 120 121 // NewKafkaBalanced creates a new KafkaBalanced input type. 122 func NewKafkaBalanced( 123 conf KafkaBalancedConfig, mgr types.Manager, log log.Modular, stats metrics.Type, 124 ) (*KafkaBalanced, error) { 125 if conf.Batching.IsNoop() { 126 conf.Batching.Count = 1 127 } 128 k := KafkaBalanced{ 129 conf: conf, 130 stats: stats, 131 groupCancelFn: func() {}, 132 log: log, 133 mgr: mgr, 134 offsets: map[string]map[int32]int64{}, 135 mRebalanced: stats.GetCounter("rebalanced"), 136 } 137 if conf.MaxBatchCount < 1 { 138 return nil, errors.New("max_batch_count must be greater than or equal to 1") 139 } 140 if conf.TLS.Enabled { 141 var err error 142 if k.tlsConf, err = conf.TLS.Get(); err != nil { 143 return nil, err 144 } 145 } 146 for _, addr := range conf.Addresses { 147 for _, splitAddr := range strings.Split(addr, ",") { 148 if trimmed := strings.TrimSpace(splitAddr); len(trimmed) > 0 { 149 k.addresses = append(k.addresses, trimmed) 150 } 151 } 152 } 153 for _, t := range conf.Topics { 154 for _, splitTopics := range strings.Split(t, ",") { 155 if trimmed := strings.TrimSpace(splitTopics); len(trimmed) > 0 { 156 k.topics = append(k.topics, trimmed) 157 } 158 } 159 } 160 if tout := conf.CommitPeriod; len(tout) > 0 { 161 var err error 162 if k.commitPeriod, err = time.ParseDuration(tout); err != nil { 163 return nil, fmt.Errorf("failed to parse commit period string: %v", err) 164 } 165 } 166 if tout := conf.Group.SessionTimeout; len(tout) > 0 { 167 var err error 168 if k.sessionTimeout, err = time.ParseDuration(tout); err != nil { 169 return nil, fmt.Errorf("failed to parse session timeout string: %v", err) 170 } 171 } 172 if tout := conf.Group.HeartbeatInterval; len(tout) > 0 { 173 var err error 174 if k.heartbeatInterval, err = time.ParseDuration(tout); err != nil { 175 return nil, fmt.Errorf("failed to parse heartbeat interval string: %v", err) 176 } 177 } 178 if tout := conf.Group.RebalanceTimeout; len(tout) > 0 { 179 var err error 180 if k.rebalanceTimeout, err = time.ParseDuration(tout); err != nil { 181 return nil, fmt.Errorf("failed to parse rebalance timeout string: %v", err) 182 } 183 } 184 if tout := conf.MaxProcessingPeriod; len(tout) > 0 { 185 var err error 186 if k.maxProcPeriod, err = time.ParseDuration(tout); err != nil { 187 return nil, fmt.Errorf("failed to parse max processing period string: %v", err) 188 } 189 } 190 191 var err error 192 if k.version, err = sarama.ParseKafkaVersion(conf.TargetVersion); err != nil { 193 return nil, err 194 } 195 return &k, nil 196 } 197 198 //------------------------------------------------------------------------------ 199 200 // Setup is run at the beginning of a new session, before ConsumeClaim. 201 func (k *KafkaBalanced) Setup(sesh sarama.ConsumerGroupSession) error { 202 k.cMut.Lock() 203 k.session = sesh 204 k.cMut.Unlock() 205 k.mRebalanced.Incr(1) 206 return nil 207 } 208 209 // Cleanup is run at the end of a session, once all ConsumeClaim goroutines have 210 // exited but before the offsets are committed for the very last time. 211 func (k *KafkaBalanced) Cleanup(sesh sarama.ConsumerGroupSession) error { 212 k.cMut.Lock() 213 k.session = nil 214 k.cMut.Unlock() 215 return nil 216 } 217 218 // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages(). 219 // Once the Messages() channel is closed, the Handler must finish its processing 220 // loop and exit. 221 func (k *KafkaBalanced) ConsumeClaim(sess sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error { 222 k.log.Debugf("Consuming messages from topic '%v' partition '%v'\n", claim.Topic(), claim.Partition()) 223 for { 224 select { 225 case msg, open := <-claim.Messages(): 226 if !open { 227 return nil 228 } 229 select { 230 case k.msgChan <- consumerMessage{ 231 ConsumerMessage: msg, 232 highWaterMark: claim.HighWaterMarkOffset(), 233 }: 234 case <-sess.Context().Done(): 235 k.log.Debugf("Stopped consuming messages from topic '%v' partition '%v'\n", claim.Topic(), claim.Partition()) 236 return nil 237 } 238 case <-sess.Context().Done(): 239 k.log.Debugf("Stopped consuming messages from topic '%v' partition '%v'\n", claim.Topic(), claim.Partition()) 240 return nil 241 } 242 } 243 } 244 245 //------------------------------------------------------------------------------ 246 247 func (k *KafkaBalanced) setOffset(topic string, partition int32, offset int64) { 248 var topicMap map[int32]int64 249 var exists bool 250 if topicMap, exists = k.offsets[topic]; !exists { 251 topicMap = map[int32]int64{} 252 k.offsets[topic] = topicMap 253 } 254 topicMap[partition] = offset 255 } 256 257 func (k *KafkaBalanced) closeGroup() { 258 k.cMut.Lock() 259 cancelFn := k.groupCancelFn 260 k.cMut.Unlock() 261 262 if cancelFn != nil { 263 cancelFn() 264 } 265 } 266 267 //------------------------------------------------------------------------------ 268 269 // Connect establishes a KafkaBalanced connection. 270 func (k *KafkaBalanced) Connect() error { 271 k.cMut.Lock() 272 defer k.cMut.Unlock() 273 if k.msgChan != nil { 274 return nil 275 } 276 277 config := sarama.NewConfig() 278 config.ClientID = k.conf.ClientID 279 config.RackID = k.conf.RackID 280 config.Net.DialTimeout = time.Second 281 config.Version = k.version 282 config.Consumer.Return.Errors = true 283 config.Consumer.MaxProcessingTime = k.maxProcPeriod 284 config.Consumer.Offsets.AutoCommit.Enable = true 285 config.Consumer.Offsets.AutoCommit.Interval = k.commitPeriod 286 config.Consumer.Group.Session.Timeout = k.sessionTimeout 287 config.Consumer.Group.Heartbeat.Interval = k.heartbeatInterval 288 config.Consumer.Group.Rebalance.Timeout = k.rebalanceTimeout 289 config.ChannelBufferSize = k.conf.FetchBufferCap 290 291 if config.Net.ReadTimeout <= k.sessionTimeout { 292 config.Net.ReadTimeout = k.sessionTimeout * 2 293 } 294 if config.Net.ReadTimeout <= k.rebalanceTimeout { 295 config.Net.ReadTimeout = k.rebalanceTimeout * 2 296 } 297 298 config.Net.TLS.Enable = k.conf.TLS.Enabled 299 if k.conf.TLS.Enabled { 300 config.Net.TLS.Config = k.tlsConf 301 } 302 if k.conf.StartFromOldest { 303 config.Consumer.Offsets.Initial = sarama.OffsetOldest 304 } 305 306 if err := k.conf.SASL.Apply(k.mgr, config); err != nil { 307 return err 308 } 309 310 // Start a new consumer group 311 group, err := sarama.NewConsumerGroup(k.addresses, k.conf.ConsumerGroup, config) 312 if err != nil { 313 return err 314 } 315 316 // Handle errors 317 go func() { 318 for { 319 gerr, open := <-group.Errors() 320 if !open { 321 return 322 } 323 if gerr != nil { 324 k.log.Errorf("KafkaBalanced message recv error: %v\n", gerr) 325 if cerr, ok := gerr.(*sarama.ConsumerError); ok { 326 if cerr.Err == sarama.ErrUnknownMemberId { 327 // Sarama doesn't seem to recover from this error. 328 go k.closeGroup() 329 } 330 } 331 } 332 } 333 }() 334 335 // Handle session 336 go func() { 337 groupLoop: 338 for { 339 ctx, doneFn := context.WithCancel(context.Background()) 340 341 k.cMut.Lock() 342 k.groupCancelFn = doneFn 343 k.cMut.Unlock() 344 345 k.log.Debugln("Starting consumer group") 346 gerr := group.Consume(ctx, k.topics, k) 347 select { 348 case <-ctx.Done(): 349 break groupLoop 350 default: 351 } 352 doneFn() 353 if gerr != nil { 354 if gerr != io.EOF { 355 k.log.Errorf("KafkaBalanced group session error: %v\n", gerr) 356 } 357 break groupLoop 358 } 359 } 360 k.log.Debugln("Closing consumer group") 361 362 group.Close() 363 364 k.cMut.Lock() 365 if k.msgChan != nil { 366 close(k.msgChan) 367 k.msgChan = nil 368 } 369 k.cMut.Unlock() 370 }() 371 372 k.msgChan = make(chan consumerMessage, k.conf.MaxBatchCount) 373 k.offsets = map[string]map[int32]int64{} 374 375 k.log.Infof("Receiving KafkaBalanced messages from addresses: %s\n", k.addresses) 376 return nil 377 } 378 379 // Read attempts to read a message from a KafkaBalanced topic. 380 func (k *KafkaBalanced) Read() (types.Message, error) { 381 k.cMut.Lock() 382 msgChan := k.msgChan 383 k.cMut.Unlock() 384 385 if msgChan == nil { 386 return nil, types.ErrNotConnected 387 } 388 389 msg := message.New(nil) 390 addPart := func(data consumerMessage) { 391 part := message.NewPart(data.Value) 392 393 meta := part.Metadata() 394 for _, hdr := range data.Headers { 395 meta.Set(string(hdr.Key), string(hdr.Value)) 396 } 397 398 lag := data.highWaterMark - data.Offset - 1 399 if lag < 0 { 400 lag = 0 401 } 402 403 meta.Set("kafka_key", string(data.Key)) 404 meta.Set("kafka_partition", strconv.Itoa(int(data.Partition))) 405 meta.Set("kafka_topic", data.Topic) 406 meta.Set("kafka_offset", strconv.Itoa(int(data.Offset))) 407 meta.Set("kafka_lag", strconv.FormatInt(lag, 10)) 408 meta.Set("kafka_timestamp_unix", strconv.FormatInt(data.Timestamp.Unix(), 10)) 409 410 msg.Append(part) 411 412 k.setOffset(data.Topic, data.Partition, data.Offset) 413 } 414 415 data, open := <-msgChan 416 if !open { 417 return nil, types.ErrNotConnected 418 } 419 addPart(data) 420 421 batchLoop: 422 for i := 1; i < k.conf.MaxBatchCount; i++ { 423 select { 424 case data, open = <-msgChan: 425 if !open { 426 return nil, types.ErrNotConnected 427 } 428 addPart(data) 429 default: 430 // Drained the buffer 431 break batchLoop 432 } 433 } 434 435 if msg.Len() == 0 { 436 return nil, types.ErrTimeout 437 } 438 return msg, nil 439 } 440 441 // Acknowledge instructs whether the current offset should be committed. 442 func (k *KafkaBalanced) Acknowledge(err error) error { 443 if err == nil { 444 k.cMut.Lock() 445 if k.session != nil { 446 for topic, v := range k.offsets { 447 for part, offset := range v { 448 k.session.MarkOffset(topic, part, offset+1, "") 449 } 450 } 451 } 452 k.cMut.Unlock() 453 } 454 return nil 455 } 456 457 // CloseAsync shuts down the KafkaBalanced input and stops processing requests. 458 func (k *KafkaBalanced) CloseAsync() { 459 go k.closeGroup() 460 } 461 462 // WaitForClose blocks until the KafkaBalanced input has closed down. 463 func (k *KafkaBalanced) WaitForClose(timeout time.Duration) error { 464 return nil 465 } 466 467 //------------------------------------------------------------------------------