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