github.com/streamdal/segmentio-kafka-go@v0.4.47-streamdal/writer.go (about) 1 package kafka 2 3 import ( 4 "bytes" 5 "context" 6 "errors" 7 "io" 8 "net" 9 "sync" 10 "sync/atomic" 11 "time" 12 13 streamdal "github.com/streamdal/streamdal/sdks/go" 14 15 metadataAPI "github.com/segmentio/kafka-go/protocol/metadata" 16 ) 17 18 // The Writer type provides the implementation of a producer of kafka messages 19 // that automatically distributes messages across partitions of a single topic 20 // using a configurable balancing policy. 21 // 22 // Writes manage the dispatch of messages across partitions of the topic they 23 // are configured to write to using a Balancer, and aggregate batches to 24 // optimize the writes to kafka. 25 // 26 // Writers may be configured to be used synchronously or asynchronously. When 27 // use synchronously, calls to WriteMessages block until the messages have been 28 // written to kafka. In this mode, the program should inspect the error returned 29 // by the function and test if it an instance of kafka.WriteErrors in order to 30 // identify which messages have succeeded or failed, for example: 31 // 32 // // Construct a synchronous writer (the default mode). 33 // w := &kafka.Writer{ 34 // Addr: kafka.TCP("localhost:9092", "localhost:9093", "localhost:9094"), 35 // Topic: "topic-A", 36 // RequiredAcks: kafka.RequireAll, 37 // } 38 // 39 // ... 40 // 41 // // Passing a context can prevent the operation from blocking indefinitely. 42 // switch err := w.WriteMessages(ctx, msgs...).(type) { 43 // case nil: 44 // case kafka.WriteErrors: 45 // for i := range msgs { 46 // if err[i] != nil { 47 // // handle the error writing msgs[i] 48 // ... 49 // } 50 // } 51 // default: 52 // // handle other errors 53 // ... 54 // } 55 // 56 // In asynchronous mode, the program may configure a completion handler on the 57 // writer to receive notifications of messages being written to kafka: 58 // 59 // w := &kafka.Writer{ 60 // Addr: kafka.TCP("localhost:9092", "localhost:9093", "localhost:9094"), 61 // Topic: "topic-A", 62 // RequiredAcks: kafka.RequireAll, 63 // Async: true, // make the writer asynchronous 64 // Completion: func(messages []kafka.Message, err error) { 65 // ... 66 // }, 67 // } 68 // 69 // ... 70 // 71 // // Because the writer is asynchronous, there is no need for the context to 72 // // be cancelled, the call will never block. 73 // if err := w.WriteMessages(context.Background(), msgs...); err != nil { 74 // // Only validation errors would be reported in this case. 75 // ... 76 // } 77 // 78 // Methods of Writer are safe to use concurrently from multiple goroutines, 79 // however the writer configuration should not be modified after first use. 80 type Writer struct { 81 // Address of the kafka cluster that this writer is configured to send 82 // messages to. 83 // 84 // This field is required, attempting to write messages to a writer with a 85 // nil address will error. 86 Addr net.Addr 87 88 // Topic is the name of the topic that the writer will produce messages to. 89 // 90 // Setting this field or not is a mutually exclusive option. If you set Topic 91 // here, you must not set Topic for any produced Message. Otherwise, if you do 92 // not set Topic, every Message must have Topic specified. 93 Topic string 94 95 // The balancer used to distribute messages across partitions. 96 // 97 // The default is to use a round-robin distribution. 98 Balancer Balancer 99 100 // Limit on how many attempts will be made to deliver a message. 101 // 102 // The default is to try at most 10 times. 103 MaxAttempts int 104 105 // WriteBackoffMin optionally sets the smallest amount of time the writer waits before 106 // it attempts to write a batch of messages 107 // 108 // Default: 100ms 109 WriteBackoffMin time.Duration 110 111 // WriteBackoffMax optionally sets the maximum amount of time the writer waits before 112 // it attempts to write a batch of messages 113 // 114 // Default: 1s 115 WriteBackoffMax time.Duration 116 117 // Limit on how many messages will be buffered before being sent to a 118 // partition. 119 // 120 // The default is to use a target batch size of 100 messages. 121 BatchSize int 122 123 // Limit the maximum size of a request in bytes before being sent to 124 // a partition. 125 // 126 // The default is to use a kafka default value of 1048576. 127 BatchBytes int64 128 129 // Time limit on how often incomplete message batches will be flushed to 130 // kafka. 131 // 132 // The default is to flush at least every second. 133 BatchTimeout time.Duration 134 135 // Timeout for read operations performed by the Writer. 136 // 137 // Defaults to 10 seconds. 138 ReadTimeout time.Duration 139 140 // Timeout for write operation performed by the Writer. 141 // 142 // Defaults to 10 seconds. 143 WriteTimeout time.Duration 144 145 // Number of acknowledges from partition replicas required before receiving 146 // a response to a produce request, the following values are supported: 147 // 148 // RequireNone (0) fire-and-forget, do not wait for acknowledgements from the 149 // RequireOne (1) wait for the leader to acknowledge the writes 150 // RequireAll (-1) wait for the full ISR to acknowledge the writes 151 // 152 // Defaults to RequireNone. 153 RequiredAcks RequiredAcks 154 155 // Setting this flag to true causes the WriteMessages method to never block. 156 // It also means that errors are ignored since the caller will not receive 157 // the returned value. Use this only if you don't care about guarantees of 158 // whether the messages were written to kafka. 159 // 160 // Defaults to false. 161 Async bool 162 163 // An optional function called when the writer succeeds or fails the 164 // delivery of messages to a kafka partition. When writing the messages 165 // fails, the `err` parameter will be non-nil. 166 // 167 // The messages that the Completion function is called with have their 168 // topic, partition, offset, and time set based on the Produce responses 169 // received from kafka. All messages passed to a call to the function have 170 // been written to the same partition. The keys and values of messages are 171 // referencing the original byte slices carried by messages in the calls to 172 // WriteMessages. 173 // 174 // The function is called from goroutines started by the writer. Calls to 175 // Close will block on the Completion function calls. When the Writer is 176 // not writing asynchronously, the WriteMessages call will also block on 177 // Completion function, which is a useful guarantee if the byte slices 178 // for the message keys and values are intended to be reused after the 179 // WriteMessages call returned. 180 // 181 // If a completion function panics, the program terminates because the 182 // panic is not recovered by the writer and bubbles up to the top of the 183 // goroutine's call stack. 184 Completion func(messages []Message, err error) 185 186 // Compression set the compression codec to be used to compress messages. 187 Compression Compression 188 189 // If not nil, specifies a logger used to report internal changes within the 190 // writer. 191 Logger Logger 192 193 // ErrorLogger is the logger used to report errors. If nil, the writer falls 194 // back to using Logger instead. 195 ErrorLogger Logger 196 197 // A transport used to send messages to kafka clusters. 198 // 199 // If nil, DefaultTransport is used. 200 Transport RoundTripper 201 202 // AllowAutoTopicCreation notifies writer to create topic if missing. 203 AllowAutoTopicCreation bool 204 205 // Manages the current set of partition-topic writers. 206 group sync.WaitGroup 207 mutex sync.Mutex 208 closed bool 209 writers map[topicPartition]*partitionWriter 210 211 // writer stats are all made of atomic values, no need for synchronization. 212 // Use a pointer to ensure 64-bit alignment of the values. The once value is 213 // used to lazily create the value when first used, allowing programs to use 214 // the zero-value value of Writer. 215 once sync.Once 216 *writerStats 217 218 // If no balancer is configured, the writer uses this one. RoundRobin values 219 // are safe to use concurrently from multiple goroutines, there is no need 220 // for extra synchronization to access this field. 221 roundRobin RoundRobin 222 223 // non-nil when a transport was created by NewWriter, remove in 1.0. 224 transport *Transport 225 226 Streamdal *streamdal.Streamdal // Streamdal addition 227 } 228 229 // WriterConfig is a configuration type used to create new instances of Writer. 230 // 231 // DEPRECATED: writer values should be configured directly by assigning their 232 // exported fields. This type is kept for backward compatibility, and will be 233 // removed in version 1.0. 234 type WriterConfig struct { 235 // The list of brokers used to discover the partitions available on the 236 // kafka cluster. 237 // 238 // This field is required, attempting to create a writer with an empty list 239 // of brokers will panic. 240 Brokers []string 241 242 // The topic that the writer will produce messages to. 243 // 244 // If provided, this will be used to set the topic for all produced messages. 245 // If not provided, each Message must specify a topic for itself. This must be 246 // mutually exclusive, otherwise the Writer will return an error. 247 Topic string 248 249 // The dialer used by the writer to establish connections to the kafka 250 // cluster. 251 // 252 // If nil, the default dialer is used instead. 253 Dialer *Dialer 254 255 // The balancer used to distribute messages across partitions. 256 // 257 // The default is to use a round-robin distribution. 258 Balancer Balancer 259 260 // Limit on how many attempts will be made to deliver a message. 261 // 262 // The default is to try at most 10 times. 263 MaxAttempts int 264 265 // DEPRECATED: in versions prior to 0.4, the writer used channels internally 266 // to dispatch messages to partitions. This has been replaced by an in-memory 267 // aggregation of batches which uses shared state instead of message passing, 268 // making this option unnecessary. 269 QueueCapacity int 270 271 // Limit on how many messages will be buffered before being sent to a 272 // partition. 273 // 274 // The default is to use a target batch size of 100 messages. 275 BatchSize int 276 277 // Limit the maximum size of a request in bytes before being sent to 278 // a partition. 279 // 280 // The default is to use a kafka default value of 1048576. 281 BatchBytes int 282 283 // Time limit on how often incomplete message batches will be flushed to 284 // kafka. 285 // 286 // The default is to flush at least every second. 287 BatchTimeout time.Duration 288 289 // Timeout for read operations performed by the Writer. 290 // 291 // Defaults to 10 seconds. 292 ReadTimeout time.Duration 293 294 // Timeout for write operation performed by the Writer. 295 // 296 // Defaults to 10 seconds. 297 WriteTimeout time.Duration 298 299 // DEPRECATED: in versions prior to 0.4, the writer used to maintain a cache 300 // the topic layout. With the change to use a transport to manage connections, 301 // the responsibility of syncing the cluster layout has been delegated to the 302 // transport. 303 RebalanceInterval time.Duration 304 305 // DEPRECATED: in versions prior to 0.4, the writer used to manage connections 306 // to the kafka cluster directly. With the change to use a transport to manage 307 // connections, the writer has no connections to manage directly anymore. 308 IdleConnTimeout time.Duration 309 310 // Number of acknowledges from partition replicas required before receiving 311 // a response to a produce request. The default is -1, which means to wait for 312 // all replicas, and a value above 0 is required to indicate how many replicas 313 // should acknowledge a message to be considered successful. 314 RequiredAcks int 315 316 // Setting this flag to true causes the WriteMessages method to never block. 317 // It also means that errors are ignored since the caller will not receive 318 // the returned value. Use this only if you don't care about guarantees of 319 // whether the messages were written to kafka. 320 Async bool 321 322 // CompressionCodec set the codec to be used to compress Kafka messages. 323 CompressionCodec 324 325 // If not nil, specifies a logger used to report internal changes within the 326 // writer. 327 Logger Logger 328 329 // ErrorLogger is the logger used to report errors. If nil, the writer falls 330 // back to using Logger instead. 331 ErrorLogger Logger 332 333 // Turn Streamdal integration on/off 334 EnableStreamdal bool 335 } 336 337 type topicPartition struct { 338 topic string 339 partition int32 340 } 341 342 // Validate method validates WriterConfig properties. 343 func (config *WriterConfig) Validate() error { 344 if len(config.Brokers) == 0 { 345 return errors.New("cannot create a kafka writer with an empty list of brokers") 346 } 347 return nil 348 } 349 350 // WriterStats is a data structure returned by a call to Writer.Stats that 351 // exposes details about the behavior of the writer. 352 type WriterStats struct { 353 Writes int64 `metric:"kafka.writer.write.count" type:"counter"` 354 Messages int64 `metric:"kafka.writer.message.count" type:"counter"` 355 Bytes int64 `metric:"kafka.writer.message.bytes" type:"counter"` 356 Errors int64 `metric:"kafka.writer.error.count" type:"counter"` 357 358 BatchTime DurationStats `metric:"kafka.writer.batch.seconds"` 359 BatchQueueTime DurationStats `metric:"kafka.writer.batch.queue.seconds"` 360 WriteTime DurationStats `metric:"kafka.writer.write.seconds"` 361 WaitTime DurationStats `metric:"kafka.writer.wait.seconds"` 362 Retries int64 `metric:"kafka.writer.retries.count" type:"counter"` 363 BatchSize SummaryStats `metric:"kafka.writer.batch.size"` 364 BatchBytes SummaryStats `metric:"kafka.writer.batch.bytes"` 365 366 MaxAttempts int64 `metric:"kafka.writer.attempts.max" type:"gauge"` 367 WriteBackoffMin time.Duration `metric:"kafka.writer.backoff.min" type:"gauge"` 368 WriteBackoffMax time.Duration `metric:"kafka.writer.backoff.max" type:"gauge"` 369 MaxBatchSize int64 `metric:"kafka.writer.batch.max" type:"gauge"` 370 BatchTimeout time.Duration `metric:"kafka.writer.batch.timeout" type:"gauge"` 371 ReadTimeout time.Duration `metric:"kafka.writer.read.timeout" type:"gauge"` 372 WriteTimeout time.Duration `metric:"kafka.writer.write.timeout" type:"gauge"` 373 RequiredAcks int64 `metric:"kafka.writer.acks.required" type:"gauge"` 374 Async bool `metric:"kafka.writer.async" type:"gauge"` 375 376 Topic string `tag:"topic"` 377 378 // DEPRECATED: these fields will only be reported for backward compatibility 379 // if the Writer was constructed with NewWriter. 380 Dials int64 `metric:"kafka.writer.dial.count" type:"counter"` 381 DialTime DurationStats `metric:"kafka.writer.dial.seconds"` 382 383 // DEPRECATED: these fields were meaningful prior to kafka-go 0.4, changes 384 // to the internal implementation and the introduction of the transport type 385 // made them unnecessary. 386 // 387 // The values will be zero but are left for backward compatibility to avoid 388 // breaking programs that used these fields. 389 Rebalances int64 390 RebalanceInterval time.Duration 391 QueueLength int64 392 QueueCapacity int64 393 ClientID string 394 } 395 396 // writerStats is a struct that contains statistics on a writer. 397 // 398 // Since atomic is used to mutate the statistics the values must be 64-bit aligned. 399 // This is easily accomplished by always allocating this struct directly, (i.e. using a pointer to the struct). 400 // See https://golang.org/pkg/sync/atomic/#pkg-note-BUG 401 type writerStats struct { 402 dials counter 403 writes counter 404 messages counter 405 bytes counter 406 errors counter 407 dialTime summary 408 batchTime summary 409 batchQueueTime summary 410 writeTime summary 411 waitTime summary 412 retries counter 413 batchSize summary 414 batchSizeBytes summary 415 } 416 417 // NewWriter creates and returns a new Writer configured with config. 418 // 419 // DEPRECATED: Writer value can be instantiated and configured directly, 420 // this function is retained for backward compatibility and will be removed 421 // in version 1.0. 422 func NewWriter(config WriterConfig) *Writer { 423 if err := config.Validate(); err != nil { 424 panic(err) 425 } 426 427 if config.Dialer == nil { 428 config.Dialer = DefaultDialer 429 } 430 431 if config.Balancer == nil { 432 config.Balancer = &RoundRobin{} 433 } 434 435 // Converts the pre-0.4 Dialer API into a Transport. 436 kafkaDialer := DefaultDialer 437 if config.Dialer != nil { 438 kafkaDialer = config.Dialer 439 } 440 441 dialer := (&net.Dialer{ 442 Timeout: kafkaDialer.Timeout, 443 Deadline: kafkaDialer.Deadline, 444 LocalAddr: kafkaDialer.LocalAddr, 445 DualStack: kafkaDialer.DualStack, 446 FallbackDelay: kafkaDialer.FallbackDelay, 447 KeepAlive: kafkaDialer.KeepAlive, 448 }) 449 450 var resolver Resolver 451 if r, ok := kafkaDialer.Resolver.(*net.Resolver); ok { 452 dialer.Resolver = r 453 } else { 454 resolver = kafkaDialer.Resolver 455 } 456 457 stats := new(writerStats) 458 // For backward compatibility with the pre-0.4 APIs, support custom 459 // resolvers by wrapping the dial function. 460 dial := func(ctx context.Context, network, addr string) (net.Conn, error) { 461 start := time.Now() 462 defer func() { 463 stats.dials.observe(1) 464 stats.dialTime.observe(int64(time.Since(start))) 465 }() 466 address, err := lookupHost(ctx, addr, resolver) 467 if err != nil { 468 return nil, err 469 } 470 return dialer.DialContext(ctx, network, address) 471 } 472 473 idleTimeout := config.IdleConnTimeout 474 if idleTimeout == 0 { 475 // Historical default value of WriterConfig.IdleTimeout, 9 minutes seems 476 // like it is way too long when there is no ping mechanism in the kafka 477 // protocol. 478 idleTimeout = 9 * time.Minute 479 } 480 481 metadataTTL := config.RebalanceInterval 482 if metadataTTL == 0 { 483 // Historical default value of WriterConfig.RebalanceInterval. 484 metadataTTL = 15 * time.Second 485 } 486 487 transport := &Transport{ 488 Dial: dial, 489 SASL: kafkaDialer.SASLMechanism, 490 TLS: kafkaDialer.TLS, 491 ClientID: kafkaDialer.ClientID, 492 IdleTimeout: idleTimeout, 493 MetadataTTL: metadataTTL, 494 } 495 496 w := &Writer{ 497 Addr: TCP(config.Brokers...), 498 Topic: config.Topic, 499 MaxAttempts: config.MaxAttempts, 500 BatchSize: config.BatchSize, 501 Balancer: config.Balancer, 502 BatchBytes: int64(config.BatchBytes), 503 BatchTimeout: config.BatchTimeout, 504 ReadTimeout: config.ReadTimeout, 505 WriteTimeout: config.WriteTimeout, 506 RequiredAcks: RequiredAcks(config.RequiredAcks), 507 Async: config.Async, 508 Logger: config.Logger, 509 ErrorLogger: config.ErrorLogger, 510 Transport: transport, 511 transport: transport, 512 writerStats: stats, 513 } 514 515 if config.RequiredAcks == 0 { 516 // Historically the writers created by NewWriter have used "all" as the 517 // default value when 0 was specified. 518 w.RequiredAcks = RequireAll 519 } 520 521 if config.CompressionCodec != nil { 522 w.Compression = Compression(config.CompressionCodec.Code()) 523 } 524 525 // Streamdal shim BEGIN 526 if config.EnableStreamdal { 527 sc, err := streamdalSetup() 528 if err != nil { 529 // NewWriter does not return error, can only panic 530 panic("unable to setup streamdal for writer: " + err.Error()) 531 } 532 533 w.Streamdal = sc 534 } 535 // Streamdal shim END 536 537 return w 538 } 539 540 // enter is called by WriteMessages to indicate that a new inflight operation 541 // has started, which helps synchronize with Close and ensure that the method 542 // does not return until all inflight operations were completed. 543 func (w *Writer) enter() bool { 544 w.mutex.Lock() 545 defer w.mutex.Unlock() 546 if w.closed { 547 return false 548 } 549 w.group.Add(1) 550 return true 551 } 552 553 // leave is called by WriteMessages to indicate that the inflight operation has 554 // completed. 555 func (w *Writer) leave() { w.group.Done() } 556 557 // spawn starts a new asynchronous operation on the writer. This method is used 558 // instead of starting goroutines inline to help manage the state of the 559 // writer's wait group. The wait group is used to block Close calls until all 560 // inflight operations have completed, therefore automatically including those 561 // started with calls to spawn. 562 func (w *Writer) spawn(f func()) { 563 w.group.Add(1) 564 go func() { 565 defer w.group.Done() 566 f() 567 }() 568 } 569 570 // Close flushes pending writes, and waits for all writes to complete before 571 // returning. Calling Close also prevents new writes from being submitted to 572 // the writer, further calls to WriteMessages and the like will fail with 573 // io.ErrClosedPipe. 574 func (w *Writer) Close() error { 575 w.mutex.Lock() 576 // Marking the writer as closed here causes future calls to WriteMessages to 577 // fail with io.ErrClosedPipe. Mutation of this field is synchronized on the 578 // writer's mutex to ensure that no more increments of the wait group are 579 // performed afterwards (which could otherwise race with the Wait below). 580 w.closed = true 581 582 // close all writers to trigger any pending batches 583 for _, writer := range w.writers { 584 writer.close() 585 } 586 587 for partition := range w.writers { 588 delete(w.writers, partition) 589 } 590 591 w.mutex.Unlock() 592 w.group.Wait() 593 594 if w.transport != nil { 595 w.transport.CloseIdleConnections() 596 } 597 598 return nil 599 } 600 601 // WriteMessages writes a batch of messages to the kafka topic configured on this 602 // writer. 603 // 604 // Unless the writer was configured to write messages asynchronously, the method 605 // blocks until all messages have been written, or until the maximum number of 606 // attempts was reached. 607 // 608 // When sending synchronously and the writer's batch size is configured to be 609 // greater than 1, this method blocks until either a full batch can be assembled 610 // or the batch timeout is reached. The batch size and timeouts are evaluated 611 // per partition, so the choice of Balancer can also influence the flushing 612 // behavior. For example, the Hash balancer will require on average N * batch 613 // size messages to trigger a flush where N is the number of partitions. The 614 // best way to achieve good batching behavior is to share one Writer amongst 615 // multiple go routines. 616 // 617 // When the method returns an error, it may be of type kafka.WriteError to allow 618 // the caller to determine the status of each message. 619 // 620 // The context passed as first argument may also be used to asynchronously 621 // cancel the operation. Note that in this case there are no guarantees made on 622 // whether messages were written to kafka, they might also still be written 623 // after this method has already returned, therefore it is important to not 624 // modify byte slices of passed messages if WriteMessages returned early due 625 // to a canceled context. 626 // The program should assume that the whole batch failed and re-write the 627 // messages later (which could then cause duplicates). 628 func (w *Writer) WriteMessages(ctx context.Context, msgs ...Message) error { 629 if w.Addr == nil { 630 return errors.New("kafka.(*Writer).WriteMessages: cannot create a kafka writer with a nil address") 631 } 632 633 if !w.enter() { 634 return io.ErrClosedPipe 635 } 636 defer w.leave() 637 638 if len(msgs) == 0 { 639 return nil 640 } 641 642 balancer := w.balancer() 643 batchBytes := w.batchBytes() 644 645 for i := range msgs { 646 n := int64(msgs[i].totalSize()) 647 if n > batchBytes { 648 // This error is left for backward compatibility with historical 649 // behavior, but it can yield O(N^2) behaviors. The expectations 650 // are that the program will check if WriteMessages returned a 651 // MessageTooLargeError, discard the message that was exceeding 652 // the maximum size, and try again. 653 return messageTooLarge(msgs, i) 654 } 655 } 656 657 // We use int32 here to half the memory footprint (compared to using int 658 // on 64 bits architectures). We map lists of the message indexes instead 659 // of the message values for the same reason, int32 is 4 bytes, vs a full 660 // Message value which is 100+ bytes and contains pointers and contributes 661 // to increasing GC work. 662 assignments := make(map[topicPartition][]int32) 663 664 newMsgs := make([]Message, 0) // Streamdal addition 665 var i int // Streamdal addition 666 for _, msg := range msgs { // Streamdal modification: underscore "i" 667 topic, err := w.chooseTopic(msg) 668 if err != nil { 669 return err 670 } 671 672 // Streamdal shim BEGIN 673 newMsg, newErr := streamdalProcess(ctx, w.Streamdal, streamdal.OperationTypeProducer, &msg, w.ErrorLogger, w.Logger) 674 if newErr != nil { 675 return errors.New("streamdal error: " + newErr.Error()) 676 } 677 678 newMsgs = append(newMsgs, *newMsg) 679 // Streamdal shim END 680 681 numPartitions, err := w.partitions(ctx, topic) 682 if err != nil { 683 return err 684 } 685 686 partition := balancer.Balance(msg, loadCachedPartitions(numPartitions)...) 687 688 key := topicPartition{ 689 topic: topic, 690 partition: int32(partition), 691 } 692 693 assignments[key] = append(assignments[key], int32(i)) 694 695 i++ // Streamdal modification 696 } 697 698 batches := w.batchMessages(newMsgs, assignments) // Streamdal modification: "newMsgs" instead of "msgs" 699 if w.Async { 700 return nil 701 } 702 703 done := ctx.Done() 704 hasErrors := false 705 for batch := range batches { 706 select { 707 case <-done: 708 return ctx.Err() 709 case <-batch.done: 710 if batch.err != nil { 711 hasErrors = true 712 } 713 } 714 } 715 716 if !hasErrors { 717 return nil 718 } 719 720 werr := make(WriteErrors, len(msgs)) 721 722 for batch, indexes := range batches { 723 for _, i := range indexes { 724 werr[i] = batch.err 725 } 726 } 727 return werr 728 } 729 730 func (w *Writer) batchMessages(messages []Message, assignments map[topicPartition][]int32) map[*writeBatch][]int32 { 731 var batches map[*writeBatch][]int32 732 if !w.Async { 733 batches = make(map[*writeBatch][]int32, len(assignments)) 734 } 735 736 w.mutex.Lock() 737 defer w.mutex.Unlock() 738 739 if w.writers == nil { 740 w.writers = map[topicPartition]*partitionWriter{} 741 } 742 743 for key, indexes := range assignments { 744 writer := w.writers[key] 745 if writer == nil { 746 writer = newPartitionWriter(w, key) 747 w.writers[key] = writer 748 } 749 wbatches := writer.writeMessages(messages, indexes) 750 751 for batch, idxs := range wbatches { 752 batches[batch] = idxs 753 } 754 } 755 756 return batches 757 } 758 759 func (w *Writer) produce(key topicPartition, batch *writeBatch) (*ProduceResponse, error) { 760 timeout := w.writeTimeout() 761 762 ctx, cancel := context.WithTimeout(context.Background(), timeout) 763 defer cancel() 764 765 return w.client(timeout).Produce(ctx, &ProduceRequest{ 766 Partition: int(key.partition), 767 Topic: key.topic, 768 RequiredAcks: w.RequiredAcks, 769 Compression: w.Compression, 770 Records: &writerRecords{ 771 msgs: batch.msgs, 772 }, 773 }) 774 } 775 776 func (w *Writer) partitions(ctx context.Context, topic string) (int, error) { 777 client := w.client(w.readTimeout()) 778 // Here we use the transport directly as an optimization to avoid the 779 // construction of temporary request and response objects made by the 780 // (*Client).Metadata API. 781 // 782 // It is expected that the transport will optimize this request by 783 // caching recent results (the kafka.Transport types does). 784 r, err := client.transport().RoundTrip(ctx, client.Addr, &metadataAPI.Request{ 785 TopicNames: []string{topic}, 786 AllowAutoTopicCreation: w.AllowAutoTopicCreation, 787 }) 788 if err != nil { 789 return 0, err 790 } 791 for _, t := range r.(*metadataAPI.Response).Topics { 792 if t.Name == topic { 793 // This should always hit, unless kafka has a bug. 794 if t.ErrorCode != 0 { 795 return 0, Error(t.ErrorCode) 796 } 797 return len(t.Partitions), nil 798 } 799 } 800 return 0, UnknownTopicOrPartition 801 } 802 803 func (w *Writer) client(timeout time.Duration) *Client { 804 return &Client{ 805 Addr: w.Addr, 806 Transport: w.Transport, 807 Timeout: timeout, 808 } 809 } 810 811 func (w *Writer) balancer() Balancer { 812 if w.Balancer != nil { 813 return w.Balancer 814 } 815 return &w.roundRobin 816 } 817 818 func (w *Writer) maxAttempts() int { 819 if w.MaxAttempts > 0 { 820 return w.MaxAttempts 821 } 822 // TODO: this is a very high default, if something has failed 9 times it 823 // seems unlikely it will succeed on the 10th attempt. However, it does 824 // carry the risk to greatly increase the volume of requests sent to the 825 // kafka cluster. We should consider reducing this default (3?). 826 return 10 827 } 828 829 func (w *Writer) writeBackoffMin() time.Duration { 830 if w.WriteBackoffMin > 0 { 831 return w.WriteBackoffMin 832 } 833 return 100 * time.Millisecond 834 } 835 836 func (w *Writer) writeBackoffMax() time.Duration { 837 if w.WriteBackoffMax > 0 { 838 return w.WriteBackoffMax 839 } 840 return 1 * time.Second 841 } 842 843 func (w *Writer) batchSize() int { 844 if w.BatchSize > 0 { 845 return w.BatchSize 846 } 847 return 100 848 } 849 850 func (w *Writer) batchBytes() int64 { 851 if w.BatchBytes > 0 { 852 return w.BatchBytes 853 } 854 return 1048576 855 } 856 857 func (w *Writer) batchTimeout() time.Duration { 858 if w.BatchTimeout > 0 { 859 return w.BatchTimeout 860 } 861 return 1 * time.Second 862 } 863 864 func (w *Writer) readTimeout() time.Duration { 865 if w.ReadTimeout > 0 { 866 return w.ReadTimeout 867 } 868 return 10 * time.Second 869 } 870 871 func (w *Writer) writeTimeout() time.Duration { 872 if w.WriteTimeout > 0 { 873 return w.WriteTimeout 874 } 875 return 10 * time.Second 876 } 877 878 func (w *Writer) withLogger(do func(Logger)) { 879 if w.Logger != nil { 880 do(w.Logger) 881 } 882 } 883 884 func (w *Writer) withErrorLogger(do func(Logger)) { 885 if w.ErrorLogger != nil { 886 do(w.ErrorLogger) 887 } else { 888 w.withLogger(do) 889 } 890 } 891 892 func (w *Writer) stats() *writerStats { 893 w.once.Do(func() { 894 // This field is not nil when the writer was constructed with NewWriter 895 // to share the value with the dial function and count dials. 896 if w.writerStats == nil { 897 w.writerStats = new(writerStats) 898 } 899 }) 900 return w.writerStats 901 } 902 903 // Stats returns a snapshot of the writer stats since the last time the method 904 // was called, or since the writer was created if it is called for the first 905 // time. 906 // 907 // A typical use of this method is to spawn a goroutine that will periodically 908 // call Stats on a kafka writer and report the metrics to a stats collection 909 // system. 910 func (w *Writer) Stats() WriterStats { 911 stats := w.stats() 912 return WriterStats{ 913 Dials: stats.dials.snapshot(), 914 Writes: stats.writes.snapshot(), 915 Messages: stats.messages.snapshot(), 916 Bytes: stats.bytes.snapshot(), 917 Errors: stats.errors.snapshot(), 918 DialTime: stats.dialTime.snapshotDuration(), 919 BatchTime: stats.batchTime.snapshotDuration(), 920 BatchQueueTime: stats.batchQueueTime.snapshotDuration(), 921 WriteTime: stats.writeTime.snapshotDuration(), 922 WaitTime: stats.waitTime.snapshotDuration(), 923 Retries: stats.retries.snapshot(), 924 BatchSize: stats.batchSize.snapshot(), 925 BatchBytes: stats.batchSizeBytes.snapshot(), 926 MaxAttempts: int64(w.maxAttempts()), 927 WriteBackoffMin: w.writeBackoffMin(), 928 WriteBackoffMax: w.writeBackoffMax(), 929 MaxBatchSize: int64(w.batchSize()), 930 BatchTimeout: w.batchTimeout(), 931 ReadTimeout: w.readTimeout(), 932 WriteTimeout: w.writeTimeout(), 933 RequiredAcks: int64(w.RequiredAcks), 934 Async: w.Async, 935 Topic: w.Topic, 936 } 937 } 938 939 func (w *Writer) chooseTopic(msg Message) (string, error) { 940 // w.Topic and msg.Topic are mutually exclusive, meaning only 1 must be set 941 // otherwise we will return an error. 942 if w.Topic != "" && msg.Topic != "" { 943 return "", errors.New("kafka.(*Writer): Topic must not be specified for both Writer and Message") 944 } else if w.Topic == "" && msg.Topic == "" { 945 return "", errors.New("kafka.(*Writer): Topic must be specified for Writer or Message") 946 } 947 948 // now we choose the topic, depending on which one is not empty 949 if msg.Topic != "" { 950 return msg.Topic, nil 951 } 952 953 return w.Topic, nil 954 } 955 956 type batchQueue struct { 957 queue []*writeBatch 958 959 // Pointers are used here to make `go vet` happy, and avoid copying mutexes. 960 // It may be better to revert these to non-pointers and avoid the copies in 961 // a different way. 962 mutex *sync.Mutex 963 cond *sync.Cond 964 965 closed bool 966 } 967 968 func (b *batchQueue) Put(batch *writeBatch) bool { 969 b.cond.L.Lock() 970 defer b.cond.L.Unlock() 971 defer b.cond.Broadcast() 972 973 if b.closed { 974 return false 975 } 976 b.queue = append(b.queue, batch) 977 return true 978 } 979 980 func (b *batchQueue) Get() *writeBatch { 981 b.cond.L.Lock() 982 defer b.cond.L.Unlock() 983 984 for len(b.queue) == 0 && !b.closed { 985 b.cond.Wait() 986 } 987 988 if len(b.queue) == 0 { 989 return nil 990 } 991 992 batch := b.queue[0] 993 b.queue[0] = nil 994 b.queue = b.queue[1:] 995 996 return batch 997 } 998 999 func (b *batchQueue) Close() { 1000 b.cond.L.Lock() 1001 defer b.cond.L.Unlock() 1002 defer b.cond.Broadcast() 1003 1004 b.closed = true 1005 } 1006 1007 func newBatchQueue(initialSize int) batchQueue { 1008 bq := batchQueue{ 1009 queue: make([]*writeBatch, 0, initialSize), 1010 mutex: &sync.Mutex{}, 1011 cond: &sync.Cond{}, 1012 } 1013 1014 bq.cond.L = bq.mutex 1015 1016 return bq 1017 } 1018 1019 // partitionWriter is a writer for a topic-partion pair. It maintains messaging order 1020 // across batches of messages. 1021 type partitionWriter struct { 1022 meta topicPartition 1023 queue batchQueue 1024 1025 mutex sync.Mutex 1026 currBatch *writeBatch 1027 1028 // reference to the writer that owns this batch. Used for the produce logic 1029 // as well as stat tracking 1030 w *Writer 1031 } 1032 1033 func newPartitionWriter(w *Writer, key topicPartition) *partitionWriter { 1034 writer := &partitionWriter{ 1035 meta: key, 1036 queue: newBatchQueue(10), 1037 w: w, 1038 } 1039 w.spawn(writer.writeBatches) 1040 return writer 1041 } 1042 1043 func (ptw *partitionWriter) writeBatches() { 1044 for { 1045 batch := ptw.queue.Get() 1046 1047 // The only time we can return nil is when the queue is closed 1048 // and empty. If the queue is closed that means 1049 // the Writer is closed so once we're here it's time to exit. 1050 if batch == nil { 1051 return 1052 } 1053 1054 ptw.writeBatch(batch) 1055 } 1056 } 1057 1058 func (ptw *partitionWriter) writeMessages(msgs []Message, indexes []int32) map[*writeBatch][]int32 { 1059 ptw.mutex.Lock() 1060 defer ptw.mutex.Unlock() 1061 1062 batchSize := ptw.w.batchSize() 1063 batchBytes := ptw.w.batchBytes() 1064 1065 var batches map[*writeBatch][]int32 1066 if !ptw.w.Async { 1067 batches = make(map[*writeBatch][]int32, 1) 1068 } 1069 1070 for _, i := range indexes { 1071 assignMessage: 1072 batch := ptw.currBatch 1073 if batch == nil { 1074 batch = ptw.newWriteBatch() 1075 ptw.currBatch = batch 1076 } 1077 if !batch.add(msgs[i], batchSize, batchBytes) { 1078 batch.trigger() 1079 ptw.queue.Put(batch) 1080 ptw.currBatch = nil 1081 goto assignMessage 1082 } 1083 1084 if batch.full(batchSize, batchBytes) { 1085 batch.trigger() 1086 ptw.queue.Put(batch) 1087 ptw.currBatch = nil 1088 } 1089 1090 if !ptw.w.Async { 1091 batches[batch] = append(batches[batch], i) 1092 } 1093 } 1094 return batches 1095 } 1096 1097 // ptw.w can be accessed here because this is called with the lock ptw.mutex already held. 1098 func (ptw *partitionWriter) newWriteBatch() *writeBatch { 1099 batch := newWriteBatch(time.Now(), ptw.w.batchTimeout()) 1100 ptw.w.spawn(func() { ptw.awaitBatch(batch) }) 1101 return batch 1102 } 1103 1104 // awaitBatch waits for a batch to either fill up or time out. 1105 // If the batch is full it only stops the timer, if the timer 1106 // expires it will queue the batch for writing if needed. 1107 func (ptw *partitionWriter) awaitBatch(batch *writeBatch) { 1108 select { 1109 case <-batch.timer.C: 1110 ptw.mutex.Lock() 1111 // detach the batch from the writer if we're still attached 1112 // and queue for writing. 1113 // Only the current batch can expire, all previous batches were already written to the queue. 1114 // If writeMesseages locks pw.mutex after the timer fires but before this goroutine 1115 // can lock pw.mutex it will either have filled the batch and enqueued it which will mean 1116 // pw.currBatch != batch so we just move on. 1117 // Otherwise, we detach the batch from the ptWriter and enqueue it for writing. 1118 if ptw.currBatch == batch { 1119 ptw.queue.Put(batch) 1120 ptw.currBatch = nil 1121 } 1122 ptw.mutex.Unlock() 1123 case <-batch.ready: 1124 // The batch became full, it was removed from the ptwriter and its 1125 // ready channel was closed. We need to close the timer to avoid 1126 // having it leak until it expires. 1127 batch.timer.Stop() 1128 } 1129 stats := ptw.w.stats() 1130 stats.batchQueueTime.observe(int64(time.Since(batch.time))) 1131 } 1132 1133 func (ptw *partitionWriter) writeBatch(batch *writeBatch) { 1134 stats := ptw.w.stats() 1135 stats.batchTime.observe(int64(time.Since(batch.time))) 1136 stats.batchSize.observe(int64(len(batch.msgs))) 1137 stats.batchSizeBytes.observe(batch.bytes) 1138 1139 var res *ProduceResponse 1140 var err error 1141 key := ptw.meta 1142 for attempt, maxAttempts := 0, ptw.w.maxAttempts(); attempt < maxAttempts; attempt++ { 1143 if attempt != 0 { 1144 stats.retries.observe(1) 1145 // TODO: should there be a way to asynchronously cancel this 1146 // operation? 1147 // 1148 // * If all goroutines that added message to this batch have stopped 1149 // waiting for it, should we abort? 1150 // 1151 // * If the writer has been closed? It reduces the durability 1152 // guarantees to abort, but may be better to avoid long wait times 1153 // on close. 1154 // 1155 delay := backoff(attempt, ptw.w.writeBackoffMin(), ptw.w.writeBackoffMax()) 1156 ptw.w.withLogger(func(log Logger) { 1157 log.Printf("backing off %s writing %d messages to %s (partition: %d)", delay, len(batch.msgs), key.topic, key.partition) 1158 }) 1159 time.Sleep(delay) 1160 } 1161 1162 ptw.w.withLogger(func(log Logger) { 1163 log.Printf("writing %d messages to %s (partition: %d)", len(batch.msgs), key.topic, key.partition) 1164 }) 1165 1166 start := time.Now() 1167 res, err = ptw.w.produce(key, batch) 1168 1169 stats.writes.observe(1) 1170 stats.messages.observe(int64(len(batch.msgs))) 1171 stats.bytes.observe(batch.bytes) 1172 // stats.writeTime used to report the duration of WriteMessages, but the 1173 // implementation was broken and reporting values in the nanoseconds 1174 // range. In kafka-go 0.4, we recylced this value to instead report the 1175 // duration of produce requests, and changed the stats.waitTime value to 1176 // report the time that kafka has throttled the requests for. 1177 stats.writeTime.observe(int64(time.Since(start))) 1178 1179 if res != nil { 1180 err = res.Error 1181 stats.waitTime.observe(int64(res.Throttle)) 1182 } 1183 1184 if err == nil { 1185 break 1186 } 1187 1188 stats.errors.observe(1) 1189 1190 ptw.w.withErrorLogger(func(log Logger) { 1191 log.Printf("error writing messages to %s (partition %d, attempt %d): %s", key.topic, key.partition, attempt, err) 1192 }) 1193 1194 if !isTemporary(err) && !isTransientNetworkError(err) { 1195 break 1196 } 1197 } 1198 1199 if res != nil { 1200 for i := range batch.msgs { 1201 m := &batch.msgs[i] 1202 m.Topic = key.topic 1203 m.Partition = int(key.partition) 1204 m.Offset = res.BaseOffset + int64(i) 1205 1206 if m.Time.IsZero() { 1207 m.Time = res.LogAppendTime 1208 } 1209 } 1210 } 1211 1212 if ptw.w.Completion != nil { 1213 ptw.w.Completion(batch.msgs, err) 1214 } 1215 1216 batch.complete(err) 1217 } 1218 1219 func (ptw *partitionWriter) close() { 1220 ptw.mutex.Lock() 1221 defer ptw.mutex.Unlock() 1222 1223 if ptw.currBatch != nil { 1224 batch := ptw.currBatch 1225 ptw.queue.Put(batch) 1226 ptw.currBatch = nil 1227 batch.trigger() 1228 } 1229 1230 ptw.queue.Close() 1231 } 1232 1233 type writeBatch struct { 1234 time time.Time 1235 msgs []Message 1236 size int 1237 bytes int64 1238 ready chan struct{} 1239 done chan struct{} 1240 timer *time.Timer 1241 err error // result of the batch completion 1242 } 1243 1244 func newWriteBatch(now time.Time, timeout time.Duration) *writeBatch { 1245 return &writeBatch{ 1246 time: now, 1247 ready: make(chan struct{}), 1248 done: make(chan struct{}), 1249 timer: time.NewTimer(timeout), 1250 } 1251 } 1252 1253 func (b *writeBatch) add(msg Message, maxSize int, maxBytes int64) bool { 1254 bytes := int64(msg.totalSize()) 1255 1256 if b.size > 0 && (b.bytes+bytes) > maxBytes { 1257 return false 1258 } 1259 1260 if cap(b.msgs) == 0 { 1261 b.msgs = make([]Message, 0, maxSize) 1262 } 1263 1264 b.msgs = append(b.msgs, msg) 1265 b.size++ 1266 b.bytes += bytes 1267 return true 1268 } 1269 1270 func (b *writeBatch) full(maxSize int, maxBytes int64) bool { 1271 return b.size >= maxSize || b.bytes >= maxBytes 1272 } 1273 1274 func (b *writeBatch) trigger() { 1275 close(b.ready) 1276 } 1277 1278 func (b *writeBatch) complete(err error) { 1279 b.err = err 1280 close(b.done) 1281 } 1282 1283 type writerRecords struct { 1284 msgs []Message 1285 index int 1286 record Record 1287 key bytesReadCloser 1288 value bytesReadCloser 1289 } 1290 1291 func (r *writerRecords) ReadRecord() (*Record, error) { 1292 if r.index >= 0 && r.index < len(r.msgs) { 1293 m := &r.msgs[r.index] 1294 r.index++ 1295 r.record = Record{ 1296 Time: m.Time, 1297 Headers: m.Headers, 1298 } 1299 if m.Key != nil { 1300 r.key.Reset(m.Key) 1301 r.record.Key = &r.key 1302 } 1303 if m.Value != nil { 1304 r.value.Reset(m.Value) 1305 r.record.Value = &r.value 1306 } 1307 return &r.record, nil 1308 } 1309 return nil, io.EOF 1310 } 1311 1312 type bytesReadCloser struct{ bytes.Reader } 1313 1314 func (*bytesReadCloser) Close() error { return nil } 1315 1316 // A cache of []int values passed to balancers of writers, used to amortize the 1317 // heap allocation of the partition index lists. 1318 // 1319 // With hindsight, the use of `...int` to pass the partition list to Balancers 1320 // was not the best design choice: kafka partition numbers are monotonically 1321 // increasing, we could have simply passed the number of partitions instead. 1322 // If we ever revisit this API, we can hopefully remove this cache. 1323 var partitionsCache atomic.Value 1324 1325 func loadCachedPartitions(numPartitions int) []int { 1326 partitions, ok := partitionsCache.Load().([]int) 1327 if ok && len(partitions) >= numPartitions { 1328 return partitions[:numPartitions] 1329 } 1330 1331 const alignment = 128 1332 n := ((numPartitions / alignment) + 1) * alignment 1333 1334 partitions = make([]int, n) 1335 for i := range partitions { 1336 partitions[i] = i 1337 } 1338 1339 partitionsCache.Store(partitions) 1340 return partitions[:numPartitions] 1341 }