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