github.com/streamdal/segmentio-kafka-go@v0.4.47-streamdal/writer_test.go (about)

     1  package kafka
     2  
     3  import (
     4  	"context"
     5  	"errors"
     6  	"fmt"
     7  	"io"
     8  	"math"
     9  	"strconv"
    10  	"strings"
    11  	"sync"
    12  	"testing"
    13  	"time"
    14  
    15  	"github.com/segmentio/kafka-go/sasl/plain"
    16  )
    17  
    18  func TestBatchQueue(t *testing.T) {
    19  	tests := []struct {
    20  		scenario string
    21  		function func(*testing.T)
    22  	}{
    23  		{
    24  			scenario: "the remaining items in a queue can be gotten after closing",
    25  			function: testBatchQueueGetWorksAfterClose,
    26  		},
    27  		{
    28  			scenario: "putting into a closed queue fails",
    29  			function: testBatchQueuePutAfterCloseFails,
    30  		},
    31  		{
    32  			scenario: "putting into a queue awakes a goroutine in a get call",
    33  			function: testBatchQueuePutWakesSleepingGetter,
    34  		},
    35  	}
    36  
    37  	for _, test := range tests {
    38  		testFunc := test.function
    39  		t.Run(test.scenario, func(t *testing.T) {
    40  			t.Parallel()
    41  			testFunc(t)
    42  		})
    43  	}
    44  }
    45  
    46  func testBatchQueuePutWakesSleepingGetter(t *testing.T) {
    47  	bq := newBatchQueue(10)
    48  	var wg sync.WaitGroup
    49  	ready := make(chan struct{})
    50  	var batch *writeBatch
    51  	wg.Add(1)
    52  	go func() {
    53  		defer wg.Done()
    54  		close(ready)
    55  		batch = bq.Get()
    56  	}()
    57  	<-ready
    58  	bq.Put(newWriteBatch(time.Now(), time.Hour*100))
    59  	wg.Wait()
    60  	if batch == nil {
    61  		t.Fatal("got nil batch")
    62  	}
    63  }
    64  
    65  func testBatchQueuePutAfterCloseFails(t *testing.T) {
    66  	bq := newBatchQueue(10)
    67  	bq.Close()
    68  	if put := bq.Put(newWriteBatch(time.Now(), time.Hour*100)); put {
    69  		t.Fatal("put batch into closed queue")
    70  	}
    71  }
    72  
    73  func testBatchQueueGetWorksAfterClose(t *testing.T) {
    74  	bq := newBatchQueue(10)
    75  	enqueueBatches := []*writeBatch{
    76  		newWriteBatch(time.Now(), time.Hour*100),
    77  		newWriteBatch(time.Now(), time.Hour*100),
    78  	}
    79  
    80  	for _, batch := range enqueueBatches {
    81  		put := bq.Put(batch)
    82  		if !put {
    83  			t.Fatal("failed to put batch into queue")
    84  		}
    85  	}
    86  
    87  	bq.Close()
    88  
    89  	batchesGotten := 0
    90  	for batchesGotten != 2 {
    91  		dequeueBatch := bq.Get()
    92  		if dequeueBatch == nil {
    93  			t.Fatalf("no batch returned from get")
    94  		}
    95  		batchesGotten++
    96  	}
    97  }
    98  
    99  func TestWriter(t *testing.T) {
   100  	tests := []struct {
   101  		scenario string
   102  		function func(*testing.T)
   103  	}{
   104  		{
   105  			scenario: "closing a writer right after creating it returns promptly with no error",
   106  			function: testWriterClose,
   107  		},
   108  
   109  		{
   110  			scenario: "writing 1 message through a writer using round-robin balancing produces 1 message to the first partition",
   111  			function: testWriterRoundRobin1,
   112  		},
   113  
   114  		{
   115  			scenario: "running out of max attempts should return an error",
   116  			function: testWriterMaxAttemptsErr,
   117  		},
   118  
   119  		{
   120  			scenario: "writing a message larger then the max bytes should return an error",
   121  			function: testWriterMaxBytes,
   122  		},
   123  
   124  		{
   125  			scenario: "writing a batch of message based on batch byte size",
   126  			function: testWriterBatchBytes,
   127  		},
   128  
   129  		{
   130  			scenario: "writing a batch of messages",
   131  			function: testWriterBatchSize,
   132  		},
   133  
   134  		{
   135  			scenario: "writing messages with a small batch byte size",
   136  			function: testWriterSmallBatchBytes,
   137  		},
   138  		{
   139  			scenario: "writing messages with headers",
   140  			function: testWriterBatchBytesHeaders,
   141  		},
   142  		{
   143  			scenario: "setting a non default balancer on the writer",
   144  			function: testWriterSetsRightBalancer,
   145  		},
   146  		{
   147  			scenario: "setting RequiredAcks to None in Writer does not cause a panic",
   148  			function: testWriterRequiredAcksNone,
   149  		},
   150  		{
   151  			scenario: "writing messages to multiple topics",
   152  			function: testWriterMultipleTopics,
   153  		},
   154  		{
   155  			scenario: "writing messages without specifying a topic",
   156  			function: testWriterMissingTopic,
   157  		},
   158  		{
   159  			scenario: "specifying topic for message when already set for writer",
   160  			function: testWriterUnexpectedMessageTopic,
   161  		},
   162  		{
   163  			scenario: "writing a message to an invalid partition",
   164  			function: testWriterInvalidPartition,
   165  		},
   166  		{
   167  			scenario: "writing a message to a non-existent topic creates the topic",
   168  			function: testWriterAutoCreateTopic,
   169  		},
   170  		{
   171  			scenario: "terminates on an attempt to write a message to a nonexistent topic",
   172  			function: testWriterTerminateMissingTopic,
   173  		},
   174  		{
   175  			scenario: "writing a message with SASL Plain authentication",
   176  			function: testWriterSasl,
   177  		},
   178  		{
   179  			scenario: "test default configuration values",
   180  			function: testWriterDefaults,
   181  		},
   182  		{
   183  			scenario: "test default stats values",
   184  			function: testWriterDefaultStats,
   185  		},
   186  		{
   187  			scenario: "test stats values with override config",
   188  			function: testWriterOverrideConfigStats,
   189  		},
   190  		{
   191  			scenario: "test write message with writer data",
   192  			function: testWriteMessageWithWriterData,
   193  		},
   194  	}
   195  
   196  	for _, test := range tests {
   197  		testFunc := test.function
   198  		t.Run(test.scenario, func(t *testing.T) {
   199  			t.Parallel()
   200  			testFunc(t)
   201  		})
   202  	}
   203  }
   204  
   205  func newTestWriter(config WriterConfig) *Writer {
   206  	if len(config.Brokers) == 0 {
   207  		config.Brokers = []string{"localhost:9092"}
   208  	}
   209  	return NewWriter(config)
   210  }
   211  
   212  func testWriterClose(t *testing.T) {
   213  	const topic = "test-writer-0"
   214  	createTopic(t, topic, 1)
   215  	defer deleteTopic(t, topic)
   216  
   217  	w := newTestWriter(WriterConfig{
   218  		Topic: topic,
   219  	})
   220  
   221  	if err := w.Close(); err != nil {
   222  		t.Error(err)
   223  	}
   224  }
   225  
   226  func testWriterRequiredAcksNone(t *testing.T) {
   227  	topic := makeTopic()
   228  	createTopic(t, topic, 1)
   229  	defer deleteTopic(t, topic)
   230  
   231  	transport := &Transport{}
   232  	defer transport.CloseIdleConnections()
   233  
   234  	writer := &Writer{
   235  		Addr:         TCP("localhost:9092"),
   236  		Topic:        topic,
   237  		Balancer:     &RoundRobin{},
   238  		RequiredAcks: RequireNone,
   239  		Transport:    transport,
   240  	}
   241  	defer writer.Close()
   242  
   243  	msg := Message{
   244  		Key:   []byte("ThisIsAKey"),
   245  		Value: []byte("Test message for required acks test"),
   246  	}
   247  
   248  	err := writer.WriteMessages(context.Background(), msg)
   249  	if err != nil {
   250  		t.Fatal(err)
   251  	}
   252  }
   253  
   254  func testWriterSetsRightBalancer(t *testing.T) {
   255  	const topic = "test-writer-1"
   256  	balancer := &CRC32Balancer{}
   257  	w := newTestWriter(WriterConfig{
   258  		Topic:    topic,
   259  		Balancer: balancer,
   260  	})
   261  	defer w.Close()
   262  
   263  	if w.Balancer != balancer {
   264  		t.Errorf("Balancer not set correctly")
   265  	}
   266  }
   267  
   268  func testWriterRoundRobin1(t *testing.T) {
   269  	const topic = "test-writer-1"
   270  	createTopic(t, topic, 1)
   271  	defer deleteTopic(t, topic)
   272  
   273  	offset, err := readOffset(topic, 0)
   274  	if err != nil {
   275  		t.Fatal(err)
   276  	}
   277  
   278  	w := newTestWriter(WriterConfig{
   279  		Topic:    topic,
   280  		Balancer: &RoundRobin{},
   281  	})
   282  	defer w.Close()
   283  
   284  	if err := w.WriteMessages(context.Background(), Message{
   285  		Value: []byte("Hello World!"),
   286  	}); err != nil {
   287  		t.Error(err)
   288  		return
   289  	}
   290  
   291  	msgs, err := readPartition(topic, 0, offset)
   292  	if err != nil {
   293  		t.Error("error reading partition", err)
   294  		return
   295  	}
   296  
   297  	if len(msgs) != 1 {
   298  		t.Error("bad messages in partition", msgs)
   299  		return
   300  	}
   301  
   302  	for _, m := range msgs {
   303  		if string(m.Value) != "Hello World!" {
   304  			t.Error("bad messages in partition", msgs)
   305  			break
   306  		}
   307  	}
   308  }
   309  
   310  func TestValidateWriter(t *testing.T) {
   311  	tests := []struct {
   312  		config       WriterConfig
   313  		errorOccured bool
   314  	}{
   315  		{config: WriterConfig{}, errorOccured: true},
   316  		{config: WriterConfig{Brokers: []string{"broker1", "broker2"}}, errorOccured: false},
   317  		{config: WriterConfig{Brokers: []string{"broker1"}, Topic: "topic1"}, errorOccured: false},
   318  	}
   319  	for _, test := range tests {
   320  		err := test.config.Validate()
   321  		if test.errorOccured && err == nil {
   322  			t.Fail()
   323  		}
   324  		if !test.errorOccured && err != nil {
   325  			t.Fail()
   326  		}
   327  	}
   328  }
   329  
   330  func testWriterMaxAttemptsErr(t *testing.T) {
   331  	topic := makeTopic()
   332  	createTopic(t, topic, 1)
   333  	defer deleteTopic(t, topic)
   334  
   335  	w := newTestWriter(WriterConfig{
   336  		Brokers:     []string{"localhost:9999"}, // nothing is listening here
   337  		Topic:       topic,
   338  		MaxAttempts: 3,
   339  		Balancer:    &RoundRobin{},
   340  	})
   341  	defer w.Close()
   342  
   343  	if err := w.WriteMessages(context.Background(), Message{
   344  		Value: []byte("Hello World!"),
   345  	}); err == nil {
   346  		t.Error("expected error")
   347  		return
   348  	}
   349  }
   350  
   351  func testWriterMaxBytes(t *testing.T) {
   352  	topic := makeTopic()
   353  	createTopic(t, topic, 1)
   354  	defer deleteTopic(t, topic)
   355  
   356  	w := newTestWriter(WriterConfig{
   357  		Topic:      topic,
   358  		BatchBytes: 25,
   359  	})
   360  	defer w.Close()
   361  
   362  	if err := w.WriteMessages(context.Background(), Message{
   363  		Value: []byte("Hi"),
   364  	}); err != nil {
   365  		t.Error(err)
   366  		return
   367  	}
   368  
   369  	firstMsg := []byte("Hello World!")
   370  	secondMsg := []byte("LeftOver!")
   371  	msgs := []Message{
   372  		{
   373  			Value: firstMsg,
   374  		},
   375  		{
   376  			Value: secondMsg,
   377  		},
   378  	}
   379  	if err := w.WriteMessages(context.Background(), msgs...); err == nil {
   380  		t.Error("expected error")
   381  		return
   382  	} else if err != nil {
   383  		var e MessageTooLargeError
   384  		switch {
   385  		case errors.As(err, &e):
   386  			if string(e.Message.Value) != string(firstMsg) {
   387  				t.Errorf("unxpected returned message. Expected: %s, Got %s", firstMsg, e.Message.Value)
   388  				return
   389  			}
   390  			if len(e.Remaining) != 1 {
   391  				t.Error("expected remaining errors; found none")
   392  				return
   393  			}
   394  			if string(e.Remaining[0].Value) != string(secondMsg) {
   395  				t.Errorf("unxpected returned message. Expected: %s, Got %s", secondMsg, e.Message.Value)
   396  				return
   397  			}
   398  
   399  		default:
   400  			t.Errorf("unexpected error: %s", err)
   401  			return
   402  		}
   403  	}
   404  }
   405  
   406  // readOffset gets the latest offset for the given topic/partition.
   407  func readOffset(topic string, partition int) (offset int64, err error) {
   408  	var conn *Conn
   409  
   410  	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   411  	defer cancel()
   412  
   413  	if conn, err = DialLeader(ctx, "tcp", "localhost:9092", topic, partition); err != nil {
   414  		err = fmt.Errorf("readOffset, DialLeader: %w", err)
   415  		return
   416  	}
   417  	defer conn.Close()
   418  
   419  	offset, err = conn.ReadLastOffset()
   420  	if err != nil {
   421  		err = fmt.Errorf("readOffset, conn.ReadLastOffset: %w", err)
   422  	}
   423  	return
   424  }
   425  
   426  func readPartition(topic string, partition int, offset int64) (msgs []Message, err error) {
   427  	var conn *Conn
   428  
   429  	if conn, err = DialLeader(context.Background(), "tcp", "localhost:9092", topic, partition); err != nil {
   430  		return
   431  	}
   432  	defer conn.Close()
   433  
   434  	conn.Seek(offset, SeekAbsolute)
   435  	conn.SetReadDeadline(time.Now().Add(10 * time.Second))
   436  	batch := conn.ReadBatch(0, 1000000000)
   437  	defer batch.Close()
   438  
   439  	for {
   440  		var msg Message
   441  
   442  		if msg, err = batch.ReadMessage(); err != nil {
   443  			if errors.Is(err, io.EOF) {
   444  				err = nil
   445  			}
   446  			return
   447  		}
   448  
   449  		msgs = append(msgs, msg)
   450  	}
   451  }
   452  
   453  func testWriterBatchBytes(t *testing.T) {
   454  	topic := makeTopic()
   455  	createTopic(t, topic, 1)
   456  	defer deleteTopic(t, topic)
   457  
   458  	offset, err := readOffset(topic, 0)
   459  	if err != nil {
   460  		t.Fatal(err)
   461  	}
   462  
   463  	w := newTestWriter(WriterConfig{
   464  		Topic:        topic,
   465  		BatchBytes:   50,
   466  		BatchTimeout: math.MaxInt32 * time.Second,
   467  		Balancer:     &RoundRobin{},
   468  	})
   469  	defer w.Close()
   470  
   471  	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   472  	defer cancel()
   473  	if err := w.WriteMessages(ctx, []Message{
   474  		{Value: []byte("M0")}, // 25 Bytes
   475  		{Value: []byte("M1")}, // 25 Bytes
   476  		{Value: []byte("M2")}, // 25 Bytes
   477  		{Value: []byte("M3")}, // 25 Bytes
   478  	}...); err != nil {
   479  		t.Error(err)
   480  		return
   481  	}
   482  
   483  	if w.Stats().Writes != 2 {
   484  		t.Error("didn't create expected batches")
   485  		return
   486  	}
   487  	msgs, err := readPartition(topic, 0, offset)
   488  	if err != nil {
   489  		t.Error("error reading partition", err)
   490  		return
   491  	}
   492  
   493  	if len(msgs) != 4 {
   494  		t.Error("bad messages in partition", msgs)
   495  		return
   496  	}
   497  
   498  	for i, m := range msgs {
   499  		if string(m.Value) == "M"+strconv.Itoa(i) {
   500  			continue
   501  		}
   502  		t.Error("bad messages in partition", string(m.Value))
   503  	}
   504  }
   505  
   506  func testWriterBatchSize(t *testing.T) {
   507  	topic := makeTopic()
   508  	createTopic(t, topic, 1)
   509  	defer deleteTopic(t, topic)
   510  
   511  	offset, err := readOffset(topic, 0)
   512  	if err != nil {
   513  		t.Fatal(err)
   514  	}
   515  
   516  	w := newTestWriter(WriterConfig{
   517  		Topic:        topic,
   518  		BatchSize:    2,
   519  		BatchTimeout: math.MaxInt32 * time.Second,
   520  		Balancer:     &RoundRobin{},
   521  	})
   522  	defer w.Close()
   523  
   524  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   525  	defer cancel()
   526  	if err := w.WriteMessages(ctx, []Message{
   527  		{Value: []byte("Hi")}, // 24 Bytes
   528  		{Value: []byte("By")}, // 24 Bytes
   529  	}...); err != nil {
   530  		t.Error(err)
   531  		return
   532  	}
   533  
   534  	if w.Stats().Writes > 1 {
   535  		t.Error("didn't batch messages")
   536  		return
   537  	}
   538  	msgs, err := readPartition(topic, 0, offset)
   539  	if err != nil {
   540  		t.Error("error reading partition", err)
   541  		return
   542  	}
   543  
   544  	if len(msgs) != 2 {
   545  		t.Error("bad messages in partition", msgs)
   546  		return
   547  	}
   548  
   549  	for _, m := range msgs {
   550  		if string(m.Value) == "Hi" || string(m.Value) == "By" {
   551  			continue
   552  		}
   553  		t.Error("bad messages in partition", msgs)
   554  	}
   555  }
   556  
   557  func testWriterSmallBatchBytes(t *testing.T) {
   558  	topic := makeTopic()
   559  	createTopic(t, topic, 1)
   560  	defer deleteTopic(t, topic)
   561  
   562  	offset, err := readOffset(topic, 0)
   563  	if err != nil {
   564  		t.Fatal(err)
   565  	}
   566  
   567  	w := newTestWriter(WriterConfig{
   568  		Topic:        topic,
   569  		BatchBytes:   25,
   570  		BatchTimeout: 50 * time.Millisecond,
   571  		Balancer:     &RoundRobin{},
   572  	})
   573  	defer w.Close()
   574  
   575  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   576  	defer cancel()
   577  	if err := w.WriteMessages(ctx, []Message{
   578  		{Value: []byte("Hi")}, // 24 Bytes
   579  		{Value: []byte("By")}, // 24 Bytes
   580  	}...); err != nil {
   581  		t.Error(err)
   582  		return
   583  	}
   584  	ws := w.Stats()
   585  	if ws.Writes != 2 {
   586  		t.Error("didn't batch messages; Writes: ", ws.Writes)
   587  		return
   588  	}
   589  	msgs, err := readPartition(topic, 0, offset)
   590  	if err != nil {
   591  		t.Error("error reading partition", err)
   592  		return
   593  	}
   594  
   595  	if len(msgs) != 2 {
   596  		t.Error("bad messages in partition", msgs)
   597  		return
   598  	}
   599  
   600  	for _, m := range msgs {
   601  		if string(m.Value) == "Hi" || string(m.Value) == "By" {
   602  			continue
   603  		}
   604  		t.Error("bad messages in partition", msgs)
   605  	}
   606  }
   607  
   608  func testWriterBatchBytesHeaders(t *testing.T) {
   609  	topic := makeTopic()
   610  	createTopic(t, topic, 1)
   611  	defer deleteTopic(t, topic)
   612  
   613  	offset, err := readOffset(topic, 0)
   614  	if err != nil {
   615  		t.Fatal(err)
   616  	}
   617  
   618  	w := newTestWriter(WriterConfig{
   619  		Topic:        topic,
   620  		BatchBytes:   100,
   621  		BatchTimeout: 50 * time.Millisecond,
   622  		Balancer:     &RoundRobin{},
   623  	})
   624  	defer w.Close()
   625  
   626  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   627  	defer cancel()
   628  	if err := w.WriteMessages(ctx, []Message{
   629  		{
   630  			Value: []byte("Hello World 1"),
   631  			Headers: []Header{
   632  				{Key: "User-Agent", Value: []byte("abc/xyz")},
   633  			},
   634  		},
   635  		{
   636  			Value: []byte("Hello World 2"),
   637  			Headers: []Header{
   638  				{Key: "User-Agent", Value: []byte("abc/xyz")},
   639  			},
   640  		},
   641  	}...); err != nil {
   642  		t.Error(err)
   643  		return
   644  	}
   645  	ws := w.Stats()
   646  	if ws.Writes != 2 {
   647  		t.Error("didn't batch messages; Writes: ", ws.Writes)
   648  		return
   649  	}
   650  	msgs, err := readPartition(topic, 0, offset)
   651  	if err != nil {
   652  		t.Error("error reading partition", err)
   653  		return
   654  	}
   655  
   656  	if len(msgs) != 2 {
   657  		t.Error("bad messages in partition", msgs)
   658  		return
   659  	}
   660  
   661  	for _, m := range msgs {
   662  		if strings.HasPrefix(string(m.Value), "Hello World") {
   663  			continue
   664  		}
   665  		t.Error("bad messages in partition", msgs)
   666  	}
   667  }
   668  
   669  func testWriterMultipleTopics(t *testing.T) {
   670  	topic1 := makeTopic()
   671  	createTopic(t, topic1, 1)
   672  	defer deleteTopic(t, topic1)
   673  
   674  	offset1, err := readOffset(topic1, 0)
   675  	if err != nil {
   676  		t.Fatal(err)
   677  	}
   678  
   679  	topic2 := makeTopic()
   680  	createTopic(t, topic2, 1)
   681  	defer deleteTopic(t, topic2)
   682  
   683  	offset2, err := readOffset(topic2, 0)
   684  	if err != nil {
   685  		t.Fatal(err)
   686  	}
   687  
   688  	w := newTestWriter(WriterConfig{
   689  		Balancer: &RoundRobin{},
   690  	})
   691  	defer w.Close()
   692  
   693  	msg1 := Message{Topic: topic1, Value: []byte("Hello")}
   694  	msg2 := Message{Topic: topic2, Value: []byte("World")}
   695  
   696  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   697  	defer cancel()
   698  	if err := w.WriteMessages(ctx, msg1, msg2); err != nil {
   699  		t.Error(err)
   700  		return
   701  	}
   702  	ws := w.Stats()
   703  	if ws.Writes != 2 {
   704  		t.Error("didn't batch messages; Writes: ", ws.Writes)
   705  		return
   706  	}
   707  
   708  	msgs1, err := readPartition(topic1, 0, offset1)
   709  	if err != nil {
   710  		t.Error("error reading partition", err)
   711  		return
   712  	}
   713  	if len(msgs1) != 1 {
   714  		t.Error("bad messages in partition", msgs1)
   715  		return
   716  	}
   717  	if string(msgs1[0].Value) != "Hello" {
   718  		t.Error("bad message in partition", msgs1)
   719  	}
   720  
   721  	msgs2, err := readPartition(topic2, 0, offset2)
   722  	if err != nil {
   723  		t.Error("error reading partition", err)
   724  		return
   725  	}
   726  	if len(msgs2) != 1 {
   727  		t.Error("bad messages in partition", msgs2)
   728  		return
   729  	}
   730  	if string(msgs2[0].Value) != "World" {
   731  		t.Error("bad message in partition", msgs2)
   732  	}
   733  }
   734  
   735  func testWriterMissingTopic(t *testing.T) {
   736  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   737  	defer cancel()
   738  
   739  	w := newTestWriter(WriterConfig{
   740  		// no topic
   741  		Balancer: &RoundRobin{},
   742  	})
   743  	defer w.Close()
   744  
   745  	msg := Message{Value: []byte("Hello World")} // no topic
   746  
   747  	if err := w.WriteMessages(ctx, msg); err == nil {
   748  		t.Error("expected error")
   749  		return
   750  	}
   751  }
   752  
   753  func testWriterInvalidPartition(t *testing.T) {
   754  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   755  	defer cancel()
   756  
   757  	topic := makeTopic()
   758  	createTopic(t, topic, 1)
   759  	defer deleteTopic(t, topic)
   760  
   761  	w := newTestWriter(WriterConfig{
   762  		Topic:       topic,
   763  		MaxAttempts: 1,                              // only try once to get the error back immediately
   764  		Balancer:    &staticBalancer{partition: -1}, // intentionally invalid partition
   765  	})
   766  	defer w.Close()
   767  
   768  	msg := Message{
   769  		Value: []byte("Hello World!"),
   770  	}
   771  
   772  	// this call should return an error and not panic (see issue #517)
   773  	if err := w.WriteMessages(ctx, msg); err == nil {
   774  		t.Fatal("expected error attempting to write message")
   775  	}
   776  }
   777  
   778  func testWriterUnexpectedMessageTopic(t *testing.T) {
   779  	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   780  	defer cancel()
   781  
   782  	topic := makeTopic()
   783  	createTopic(t, topic, 1)
   784  	defer deleteTopic(t, topic)
   785  
   786  	w := newTestWriter(WriterConfig{
   787  		Topic:    topic,
   788  		Balancer: &RoundRobin{},
   789  	})
   790  	defer w.Close()
   791  
   792  	msg := Message{Topic: "should-fail", Value: []byte("Hello World")}
   793  
   794  	if err := w.WriteMessages(ctx, msg); err == nil {
   795  		t.Error("expected error")
   796  		return
   797  	}
   798  }
   799  
   800  func testWriteMessageWithWriterData(t *testing.T) {
   801  	topic := makeTopic()
   802  	createTopic(t, topic, 1)
   803  	defer deleteTopic(t, topic)
   804  	w := newTestWriter(WriterConfig{
   805  		Topic:    topic,
   806  		Balancer: &RoundRobin{},
   807  	})
   808  	defer w.Close()
   809  
   810  	index := 0
   811  	w.Completion = func(messages []Message, err error) {
   812  		if err != nil {
   813  			t.Errorf("unexpected error %v", err)
   814  		}
   815  
   816  		for _, msg := range messages {
   817  			meta := msg.WriterData.(int)
   818  			if index != meta {
   819  				t.Errorf("metadata is not correct, index = %d, writerData = %d", index, meta)
   820  			}
   821  			index += 1
   822  		}
   823  	}
   824  
   825  	msg := Message{Key: []byte("key"), Value: []byte("Hello World")}
   826  	for i := 0; i < 5; i++ {
   827  		ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   828  		defer cancel()
   829  
   830  		msg.WriterData = i
   831  		err := w.WriteMessages(ctx, msg)
   832  		if err != nil {
   833  			t.Errorf("unexpected error %v", err)
   834  		}
   835  	}
   836  
   837  }
   838  
   839  func testWriterAutoCreateTopic(t *testing.T) {
   840  	topic := makeTopic()
   841  	// Assume it's going to get created.
   842  	defer deleteTopic(t, topic)
   843  
   844  	w := newTestWriter(WriterConfig{
   845  		Topic:    topic,
   846  		Balancer: &RoundRobin{},
   847  	})
   848  	w.AllowAutoTopicCreation = true
   849  	defer w.Close()
   850  
   851  	msg := Message{Key: []byte("key"), Value: []byte("Hello World")}
   852  
   853  	var err error
   854  	const retries = 5
   855  	for i := 0; i < retries; i++ {
   856  		ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   857  		defer cancel()
   858  		err = w.WriteMessages(ctx, msg)
   859  		if errors.Is(err, LeaderNotAvailable) || errors.Is(err, context.DeadlineExceeded) {
   860  			time.Sleep(time.Millisecond * 250)
   861  			continue
   862  		}
   863  
   864  		if err != nil {
   865  			t.Errorf("unexpected error %v", err)
   866  			return
   867  		}
   868  	}
   869  	if err != nil {
   870  		t.Errorf("unable to create topic %v", err)
   871  	}
   872  }
   873  
   874  func testWriterTerminateMissingTopic(t *testing.T) {
   875  	topic := makeTopic()
   876  
   877  	transport := &Transport{}
   878  	defer transport.CloseIdleConnections()
   879  
   880  	writer := &Writer{
   881  		Addr:                   TCP("localhost:9092"),
   882  		Topic:                  topic,
   883  		Balancer:               &RoundRobin{},
   884  		RequiredAcks:           RequireNone,
   885  		AllowAutoTopicCreation: false,
   886  		Transport:              transport,
   887  	}
   888  	defer writer.Close()
   889  
   890  	msg := Message{Value: []byte("FooBar")}
   891  
   892  	if err := writer.WriteMessages(context.Background(), msg); err == nil {
   893  		t.Fatal("Kafka error [3] UNKNOWN_TOPIC_OR_PARTITION is expected")
   894  		return
   895  	}
   896  }
   897  
   898  func testWriterSasl(t *testing.T) {
   899  	topic := makeTopic()
   900  	defer deleteTopic(t, topic)
   901  	dialer := &Dialer{
   902  		Timeout: 10 * time.Second,
   903  		SASLMechanism: plain.Mechanism{
   904  			Username: "adminplain",
   905  			Password: "admin-secret",
   906  		},
   907  	}
   908  
   909  	w := newTestWriter(WriterConfig{
   910  		Dialer:  dialer,
   911  		Topic:   topic,
   912  		Brokers: []string{"localhost:9093"},
   913  	})
   914  
   915  	w.AllowAutoTopicCreation = true
   916  
   917  	defer w.Close()
   918  
   919  	msg := Message{Key: []byte("key"), Value: []byte("Hello World")}
   920  
   921  	var err error
   922  	const retries = 5
   923  	for i := 0; i < retries; i++ {
   924  		ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   925  		defer cancel()
   926  		err = w.WriteMessages(ctx, msg)
   927  		if errors.Is(err, LeaderNotAvailable) || errors.Is(err, context.DeadlineExceeded) {
   928  			time.Sleep(time.Millisecond * 250)
   929  			continue
   930  		}
   931  
   932  		if err != nil {
   933  			t.Errorf("unexpected error %v", err)
   934  			return
   935  		}
   936  	}
   937  	if err != nil {
   938  		t.Errorf("unable to create topic %v", err)
   939  	}
   940  }
   941  
   942  func testWriterDefaults(t *testing.T) {
   943  	w := &Writer{}
   944  	defer w.Close()
   945  
   946  	if w.writeBackoffMin() != 100*time.Millisecond {
   947  		t.Error("Incorrect default min write backoff delay")
   948  	}
   949  
   950  	if w.writeBackoffMax() != 1*time.Second {
   951  		t.Error("Incorrect default max write backoff delay")
   952  	}
   953  }
   954  
   955  func testWriterDefaultStats(t *testing.T) {
   956  	w := &Writer{}
   957  	defer w.Close()
   958  
   959  	stats := w.Stats()
   960  
   961  	if stats.MaxAttempts == 0 {
   962  		t.Error("Incorrect default MaxAttempts value")
   963  	}
   964  
   965  	if stats.WriteBackoffMin == 0 {
   966  		t.Error("Incorrect default WriteBackoffMin value")
   967  	}
   968  
   969  	if stats.WriteBackoffMax == 0 {
   970  		t.Error("Incorrect default WriteBackoffMax value")
   971  	}
   972  
   973  	if stats.MaxBatchSize == 0 {
   974  		t.Error("Incorrect default MaxBatchSize value")
   975  	}
   976  
   977  	if stats.BatchTimeout == 0 {
   978  		t.Error("Incorrect default BatchTimeout value")
   979  	}
   980  
   981  	if stats.ReadTimeout == 0 {
   982  		t.Error("Incorrect default ReadTimeout value")
   983  	}
   984  
   985  	if stats.WriteTimeout == 0 {
   986  		t.Error("Incorrect default WriteTimeout value")
   987  	}
   988  }
   989  
   990  func testWriterOverrideConfigStats(t *testing.T) {
   991  	w := &Writer{
   992  		MaxAttempts:     6,
   993  		WriteBackoffMin: 2,
   994  		WriteBackoffMax: 4,
   995  		BatchSize:       1024,
   996  		BatchTimeout:    16,
   997  		ReadTimeout:     24,
   998  		WriteTimeout:    32,
   999  	}
  1000  	defer w.Close()
  1001  
  1002  	stats := w.Stats()
  1003  
  1004  	if stats.MaxAttempts != 6 {
  1005  		t.Error("Incorrect MaxAttempts value")
  1006  	}
  1007  
  1008  	if stats.WriteBackoffMin != 2 {
  1009  		t.Error("Incorrect WriteBackoffMin value")
  1010  	}
  1011  
  1012  	if stats.WriteBackoffMax != 4 {
  1013  		t.Error("Incorrect WriteBackoffMax value")
  1014  	}
  1015  
  1016  	if stats.MaxBatchSize != 1024 {
  1017  		t.Error("Incorrect MaxBatchSize value")
  1018  	}
  1019  
  1020  	if stats.BatchTimeout != 16 {
  1021  		t.Error("Incorrect BatchTimeout value")
  1022  	}
  1023  
  1024  	if stats.ReadTimeout != 24 {
  1025  		t.Error("Incorrect ReadTimeout value")
  1026  	}
  1027  
  1028  	if stats.WriteTimeout != 32 {
  1029  		t.Error("Incorrect WriteTimeout value")
  1030  	}
  1031  }
  1032  
  1033  type staticBalancer struct {
  1034  	partition int
  1035  }
  1036  
  1037  func (b *staticBalancer) Balance(_ Message, partitions ...int) int {
  1038  	return b.partition
  1039  }