github.com/Jeffail/benthos/v3@v3.65.0/lib/output/writer/kafka.go (about) 1 package writer 2 3 import ( 4 "context" 5 "crypto/tls" 6 "fmt" 7 "strconv" 8 "strings" 9 "sync" 10 "time" 11 12 batchInternal "github.com/Jeffail/benthos/v3/internal/batch" 13 "github.com/Jeffail/benthos/v3/internal/bloblang/field" 14 "github.com/Jeffail/benthos/v3/internal/interop" 15 "github.com/Jeffail/benthos/v3/internal/metadata" 16 "github.com/Jeffail/benthos/v3/lib/log" 17 "github.com/Jeffail/benthos/v3/lib/message/batch" 18 "github.com/Jeffail/benthos/v3/lib/metrics" 19 "github.com/Jeffail/benthos/v3/lib/types" 20 "github.com/Jeffail/benthos/v3/lib/util/hash/murmur2" 21 "github.com/Jeffail/benthos/v3/lib/util/kafka/sasl" 22 "github.com/Jeffail/benthos/v3/lib/util/retries" 23 btls "github.com/Jeffail/benthos/v3/lib/util/tls" 24 "github.com/Shopify/sarama" 25 "github.com/cenkalti/backoff/v4" 26 ) 27 28 //------------------------------------------------------------------------------ 29 30 // KafkaConfig contains configuration fields for the Kafka output type. 31 type KafkaConfig struct { 32 Addresses []string `json:"addresses" yaml:"addresses"` 33 ClientID string `json:"client_id" yaml:"client_id"` 34 RackID string `json:"rack_id" yaml:"rack_id"` 35 Key string `json:"key" yaml:"key"` 36 Partitioner string `json:"partitioner" yaml:"partitioner"` 37 Partition string `json:"partition" yaml:"partition"` 38 Topic string `json:"topic" yaml:"topic"` 39 Compression string `json:"compression" yaml:"compression"` 40 MaxMsgBytes int `json:"max_msg_bytes" yaml:"max_msg_bytes"` 41 Timeout string `json:"timeout" yaml:"timeout"` 42 AckReplicas bool `json:"ack_replicas" yaml:"ack_replicas"` 43 TargetVersion string `json:"target_version" yaml:"target_version"` 44 TLS btls.Config `json:"tls" yaml:"tls"` 45 SASL sasl.Config `json:"sasl" yaml:"sasl"` 46 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 47 retries.Config `json:",inline" yaml:",inline"` 48 RetryAsBatch bool `json:"retry_as_batch" yaml:"retry_as_batch"` 49 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 50 StaticHeaders map[string]string `json:"static_headers" yaml:"static_headers"` 51 Metadata metadata.ExcludeFilterConfig `json:"metadata" yaml:"metadata"` 52 InjectTracingMap string `json:"inject_tracing_map" yaml:"inject_tracing_map"` 53 54 // TODO: V4 remove this. 55 RoundRobinPartitions bool `json:"round_robin_partitions" yaml:"round_robin_partitions"` 56 } 57 58 // NewKafkaConfig creates a new KafkaConfig with default values. 59 func NewKafkaConfig() KafkaConfig { 60 rConf := retries.NewConfig() 61 rConf.Backoff.InitialInterval = "3s" 62 rConf.Backoff.MaxInterval = "10s" 63 rConf.Backoff.MaxElapsedTime = "30s" 64 65 return KafkaConfig{ 66 Addresses: []string{"localhost:9092"}, 67 ClientID: "benthos_kafka_output", 68 RackID: "", 69 Key: "", 70 RoundRobinPartitions: false, 71 Partitioner: "fnv1a_hash", 72 Partition: "", 73 Topic: "benthos_stream", 74 Compression: "none", 75 MaxMsgBytes: 1000000, 76 Timeout: "5s", 77 AckReplicas: false, 78 TargetVersion: sarama.V1_0_0_0.String(), 79 StaticHeaders: map[string]string{}, 80 Metadata: metadata.NewExcludeFilterConfig(), 81 TLS: btls.NewConfig(), 82 SASL: sasl.NewConfig(), 83 MaxInFlight: 1, 84 Config: rConf, 85 RetryAsBatch: false, 86 Batching: batch.NewPolicyConfig(), 87 } 88 } 89 90 //------------------------------------------------------------------------------ 91 92 // Kafka is a writer type that writes messages into kafka. 93 type Kafka struct { 94 log log.Modular 95 mgr types.Manager 96 stats metrics.Type 97 98 backoffCtor func() backoff.BackOff 99 100 tlsConf *tls.Config 101 timeout time.Duration 102 103 addresses []string 104 version sarama.KafkaVersion 105 conf KafkaConfig 106 107 key *field.Expression 108 topic *field.Expression 109 partition *field.Expression 110 111 producer sarama.SyncProducer 112 compression sarama.CompressionCodec 113 partitioner sarama.PartitionerConstructor 114 115 staticHeaders map[string]string 116 metaFilter *metadata.ExcludeFilter 117 118 connMut sync.RWMutex 119 } 120 121 // NewKafka creates a new Kafka writer type. 122 func NewKafka(conf KafkaConfig, mgr types.Manager, log log.Modular, stats metrics.Type) (*Kafka, error) { 123 compression, err := strToCompressionCodec(conf.Compression) 124 if err != nil { 125 return nil, err 126 } 127 128 // for backward compatitility 129 if conf.RoundRobinPartitions { 130 conf.Partitioner = "round_robin" 131 log.Warnln("The field 'round_robin_partitions' is deprecated, please use the 'partitioner' field (set to 'round_robin') instead.") 132 } 133 134 if conf.Partition == "" && conf.Partitioner == "manual" { 135 return nil, fmt.Errorf("partition field required for 'manual' partitioner") 136 } else if len(conf.Partition) > 0 && conf.Partitioner != "manual" { 137 return nil, fmt.Errorf("partition field can only be specified for 'manual' partitioner") 138 } 139 140 partitioner, err := strToPartitioner(conf.Partitioner) 141 if err != nil { 142 return nil, err 143 } 144 145 k := Kafka{ 146 log: log, 147 mgr: mgr, 148 stats: stats, 149 150 conf: conf, 151 compression: compression, 152 partitioner: partitioner, 153 staticHeaders: conf.StaticHeaders, 154 } 155 156 if k.metaFilter, err = conf.Metadata.Filter(); err != nil { 157 return nil, fmt.Errorf("failed to construct metadata filter: %w", err) 158 } 159 160 if k.key, err = interop.NewBloblangField(mgr, conf.Key); err != nil { 161 return nil, fmt.Errorf("failed to parse key expression: %v", err) 162 } 163 if k.topic, err = interop.NewBloblangField(mgr, conf.Topic); err != nil { 164 return nil, fmt.Errorf("failed to parse topic expression: %v", err) 165 } 166 if k.partition, err = interop.NewBloblangField(mgr, conf.Partition); err != nil { 167 return nil, fmt.Errorf("failed to parse parition expression: %v", err) 168 } 169 if k.backoffCtor, err = conf.Config.GetCtor(); err != nil { 170 return nil, err 171 } 172 173 if tout := conf.Timeout; len(tout) > 0 { 174 var err error 175 if k.timeout, err = time.ParseDuration(tout); err != nil { 176 return nil, fmt.Errorf("failed to parse timeout string: %v", err) 177 } 178 } 179 180 if conf.TLS.Enabled { 181 var err error 182 if k.tlsConf, err = conf.TLS.Get(); err != nil { 183 return nil, err 184 } 185 } 186 187 if k.version, err = sarama.ParseKafkaVersion(conf.TargetVersion); err != nil { 188 return nil, err 189 } 190 191 for _, addr := range conf.Addresses { 192 for _, splitAddr := range strings.Split(addr, ",") { 193 if trimmed := strings.TrimSpace(splitAddr); len(trimmed) > 0 { 194 k.addresses = append(k.addresses, trimmed) 195 } 196 } 197 } 198 199 return &k, nil 200 } 201 202 //------------------------------------------------------------------------------ 203 204 func strToCompressionCodec(str string) (sarama.CompressionCodec, error) { 205 switch str { 206 case "none": 207 return sarama.CompressionNone, nil 208 case "snappy": 209 return sarama.CompressionSnappy, nil 210 case "lz4": 211 return sarama.CompressionLZ4, nil 212 case "gzip": 213 return sarama.CompressionGZIP, nil 214 case "zstd": 215 return sarama.CompressionZSTD, nil 216 } 217 return sarama.CompressionNone, fmt.Errorf("compression codec not recognised: %v", str) 218 } 219 220 //------------------------------------------------------------------------------ 221 222 func strToPartitioner(str string) (sarama.PartitionerConstructor, error) { 223 switch str { 224 case "fnv1a_hash": 225 return sarama.NewHashPartitioner, nil 226 case "murmur2_hash": 227 return sarama.NewCustomPartitioner( 228 sarama.WithAbsFirst(), 229 sarama.WithCustomHashFunction(murmur2.New32), 230 ), nil 231 case "random": 232 return sarama.NewRandomPartitioner, nil 233 case "round_robin": 234 return sarama.NewRoundRobinPartitioner, nil 235 case "manual": 236 return sarama.NewManualPartitioner, nil 237 default: 238 } 239 return nil, fmt.Errorf("partitioner not recognised: %v", str) 240 } 241 242 //------------------------------------------------------------------------------ 243 244 func (k *Kafka) buildSystemHeaders(part types.Part) []sarama.RecordHeader { 245 if k.version.IsAtLeast(sarama.V0_11_0_0) { 246 out := []sarama.RecordHeader{} 247 k.metaFilter.Iter(part.Metadata(), func(k, v string) error { 248 out = append(out, sarama.RecordHeader{ 249 Key: []byte(k), 250 Value: []byte(v), 251 }) 252 return nil 253 }) 254 return out 255 } 256 257 // no headers before version 0.11 258 return nil 259 } 260 261 //------------------------------------------------------------------------------ 262 263 func (k *Kafka) buildUserDefinedHeaders(staticHeaders map[string]string) []sarama.RecordHeader { 264 if k.version.IsAtLeast(sarama.V0_11_0_0) { 265 out := make([]sarama.RecordHeader, 0, len(staticHeaders)) 266 267 for name, value := range staticHeaders { 268 out = append(out, sarama.RecordHeader{ 269 Key: []byte(name), 270 Value: []byte(value), 271 }) 272 } 273 274 return out 275 } 276 277 // no headers before version 0.11 278 return nil 279 } 280 281 //------------------------------------------------------------------------------ 282 283 // ConnectWithContext attempts to establish a connection to a Kafka broker. 284 func (k *Kafka) ConnectWithContext(ctx context.Context) error { 285 return k.Connect() 286 } 287 288 // Connect attempts to establish a connection to a Kafka broker. 289 func (k *Kafka) Connect() error { 290 k.connMut.Lock() 291 defer k.connMut.Unlock() 292 293 if k.producer != nil { 294 return nil 295 } 296 297 config := sarama.NewConfig() 298 config.ClientID = k.conf.ClientID 299 config.RackID = k.conf.RackID 300 301 config.Version = k.version 302 303 config.Producer.Compression = k.compression 304 config.Producer.Partitioner = k.partitioner 305 config.Producer.MaxMessageBytes = k.conf.MaxMsgBytes 306 config.Producer.Timeout = k.timeout 307 config.Producer.Return.Errors = true 308 config.Producer.Return.Successes = true 309 config.Net.TLS.Enable = k.conf.TLS.Enabled 310 if k.conf.TLS.Enabled { 311 config.Net.TLS.Config = k.tlsConf 312 } 313 if err := k.conf.SASL.Apply(k.mgr, config); err != nil { 314 return err 315 } 316 317 if k.conf.AckReplicas { 318 config.Producer.RequiredAcks = sarama.WaitForAll 319 } else { 320 config.Producer.RequiredAcks = sarama.WaitForLocal 321 } 322 323 var err error 324 k.producer, err = sarama.NewSyncProducer(k.addresses, config) 325 326 if err == nil { 327 k.log.Infof("Sending Kafka messages to addresses: %s\n", k.addresses) 328 } 329 return err 330 } 331 332 // Write will attempt to write a message to Kafka, wait for acknowledgement, and 333 // returns an error if applicable. 334 func (k *Kafka) Write(msg types.Message) error { 335 return k.WriteWithContext(context.Background(), msg) 336 } 337 338 // WriteWithContext will attempt to write a message to Kafka, wait for 339 // acknowledgement, and returns an error if applicable. 340 func (k *Kafka) WriteWithContext(ctx context.Context, msg types.Message) error { 341 k.connMut.RLock() 342 producer := k.producer 343 k.connMut.RUnlock() 344 345 if producer == nil { 346 return types.ErrNotConnected 347 } 348 349 boff := k.backoffCtor() 350 351 userDefinedHeaders := k.buildUserDefinedHeaders(k.staticHeaders) 352 msgs := []*sarama.ProducerMessage{} 353 354 err := msg.Iter(func(i int, p types.Part) error { 355 key := k.key.Bytes(i, msg) 356 nextMsg := &sarama.ProducerMessage{ 357 Topic: k.topic.String(i, msg), 358 Value: sarama.ByteEncoder(p.Get()), 359 Headers: append(k.buildSystemHeaders(p), userDefinedHeaders...), 360 Metadata: i, // Store the original index for later reference. 361 } 362 if len(key) > 0 { 363 nextMsg.Key = sarama.ByteEncoder(key) 364 } 365 366 // Only parse and set the partition if we are configured for manual 367 // partitioner. Although samara will (currently) ignore the partition 368 // field when not using a manual partitioner, we should only set it when 369 // we explicitly want that. 370 if k.conf.Partitioner == "manual" { 371 partitionString := k.partition.String(i, msg) 372 if partitionString == "" { 373 return fmt.Errorf("partition expression failed to produce a value") 374 } 375 376 partitionInt, err := strconv.Atoi(partitionString) 377 if err != nil { 378 return fmt.Errorf("failed to parse valid integer from partition expression: %w", err) 379 } 380 if partitionInt < 0 { 381 return fmt.Errorf("invalid partition parsed from expression, must be >= 0, got %v", partitionInt) 382 } 383 // samara requires a 32-bit integer for the partition field 384 nextMsg.Partition = int32(partitionInt) 385 } 386 msgs = append(msgs, nextMsg) 387 return nil 388 }) 389 390 if err != nil { 391 return err 392 } 393 394 err = producer.SendMessages(msgs) 395 for err != nil { 396 if pErrs, ok := err.(sarama.ProducerErrors); !k.conf.RetryAsBatch && ok { 397 if len(pErrs) == 0 { 398 break 399 } 400 batchErr := batchInternal.NewError(msg, pErrs[0].Err) 401 msgs = nil 402 for _, pErr := range pErrs { 403 if mIndex, ok := pErr.Msg.Metadata.(int); ok { 404 batchErr.Failed(mIndex, pErr.Err) 405 } 406 msgs = append(msgs, pErr.Msg) 407 } 408 if len(pErrs) == batchErr.IndexedErrors() { 409 err = batchErr 410 } else { 411 // If these lengths don't match then somehow we failed to obtain 412 // the indexes from metadata, which implies something is wrong 413 // with our logic here. 414 k.log.Warnln("Unable to determine batch index of errors") 415 } 416 k.log.Errorf("Failed to send '%v' messages: %v\n", len(pErrs), err) 417 } else { 418 k.log.Errorf("Failed to send messages: %v\n", err) 419 } 420 421 tNext := boff.NextBackOff() 422 if tNext == backoff.Stop { 423 return err 424 } 425 select { 426 case <-ctx.Done(): 427 return err 428 case <-time.After(tNext): 429 } 430 431 // Recheck connection is alive 432 k.connMut.RLock() 433 producer = k.producer 434 k.connMut.RUnlock() 435 436 if producer == nil { 437 return types.ErrNotConnected 438 } 439 err = producer.SendMessages(msgs) 440 } 441 442 return nil 443 } 444 445 // CloseAsync shuts down the Kafka writer and stops processing messages. 446 func (k *Kafka) CloseAsync() { 447 go func() { 448 k.connMut.Lock() 449 if k.producer != nil { 450 k.producer.Close() 451 k.producer = nil 452 } 453 k.connMut.Unlock() 454 }() 455 } 456 457 // WaitForClose blocks until the Kafka writer has closed down. 458 func (k *Kafka) WaitForClose(timeout time.Duration) error { 459 return nil 460 } 461 462 //------------------------------------------------------------------------------