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