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

     1  package kafka
     2  
     3  import (
     4  	"bytes"
     5  	"context"
     6  	"fmt"
     7  	"hash/crc32"
     8  	"testing"
     9  	"time"
    10  
    11  	ktesting "github.com/segmentio/kafka-go/testing"
    12  )
    13  
    14  const (
    15  	testCorrelationID = 1
    16  	testClientID      = "localhost"
    17  	testTopic         = "topic"
    18  	testPartition     = 42
    19  )
    20  
    21  type WriteVarIntTestCase struct {
    22  	v  []byte
    23  	tc int64
    24  }
    25  
    26  func TestWriteVarInt(t *testing.T) {
    27  	testCases := []*WriteVarIntTestCase{
    28  		{v: []byte{0}, tc: 0},
    29  		{v: []byte{2}, tc: 1},
    30  		{v: []byte{1}, tc: -1},
    31  		{v: []byte{3}, tc: -2},
    32  		{v: []byte{128, 2}, tc: 128},
    33  		{v: []byte{254, 1}, tc: 127},
    34  		{v: []byte{142, 6}, tc: 391},
    35  		{v: []byte{142, 134, 6}, tc: 49543},
    36  	}
    37  
    38  	for _, tc := range testCases {
    39  		b := &bytes.Buffer{}
    40  		w := &writeBuffer{w: b}
    41  		w.writeVarInt(tc.tc)
    42  
    43  		if !bytes.Equal(b.Bytes(), tc.v) {
    44  			t.Errorf("Expected %v; got %v", tc.v, b.Bytes())
    45  		}
    46  	}
    47  }
    48  
    49  func TestWriteOptimizations(t *testing.T) {
    50  	t.Run("writeFetchRequestV2", testWriteFetchRequestV2)
    51  	t.Run("writeListOffsetRequestV1", testWriteListOffsetRequestV1)
    52  	t.Run("writeProduceRequestV2", testWriteProduceRequestV2)
    53  }
    54  
    55  func testWriteFetchRequestV2(t *testing.T) {
    56  	const offset = 42
    57  	const minBytes = 10
    58  	const maxBytes = 1000
    59  	const maxWait = 100 * time.Millisecond
    60  	testWriteOptimization(t,
    61  		requestHeader{
    62  			ApiKey:        int16(fetch),
    63  			ApiVersion:    int16(v2),
    64  			CorrelationID: testCorrelationID,
    65  			ClientID:      testClientID,
    66  		},
    67  		fetchRequestV2{
    68  			ReplicaID:   -1,
    69  			MaxWaitTime: milliseconds(maxWait),
    70  			MinBytes:    minBytes,
    71  			Topics: []fetchRequestTopicV2{{
    72  				TopicName: testTopic,
    73  				Partitions: []fetchRequestPartitionV2{{
    74  					Partition:   testPartition,
    75  					FetchOffset: offset,
    76  					MaxBytes:    maxBytes,
    77  				}},
    78  			}},
    79  		},
    80  		func(w *writeBuffer) {
    81  			w.writeFetchRequestV2(testCorrelationID, testClientID, testTopic, testPartition, offset, minBytes, maxBytes, maxWait)
    82  		},
    83  	)
    84  }
    85  
    86  func testWriteListOffsetRequestV1(t *testing.T) {
    87  	const time = -1
    88  	testWriteOptimization(t,
    89  		requestHeader{
    90  			ApiKey:        int16(listOffsets),
    91  			ApiVersion:    int16(v1),
    92  			CorrelationID: testCorrelationID,
    93  			ClientID:      testClientID,
    94  		},
    95  		listOffsetRequestV1{
    96  			ReplicaID: -1,
    97  			Topics: []listOffsetRequestTopicV1{{
    98  				TopicName: testTopic,
    99  				Partitions: []listOffsetRequestPartitionV1{{
   100  					Partition: testPartition,
   101  					Time:      time,
   102  				}},
   103  			}},
   104  		},
   105  		func(w *writeBuffer) {
   106  			w.writeListOffsetRequestV1(testCorrelationID, testClientID, testTopic, testPartition, time)
   107  		},
   108  	)
   109  }
   110  
   111  func testWriteProduceRequestV2(t *testing.T) {
   112  	key := []byte(nil)
   113  	val := []byte("Hello World!")
   114  
   115  	msg := messageSetItem{
   116  		Offset: 10,
   117  		Message: message{
   118  			MagicByte:  1,
   119  			Attributes: 0,
   120  			Key:        key,
   121  			Value:      val,
   122  		},
   123  	}
   124  	msg.MessageSize = msg.Message.size()
   125  	msg.Message.CRC = msg.Message.crc32(&crc32Writer{
   126  		table: crc32.IEEETable,
   127  	})
   128  
   129  	const timeout = 100
   130  	testWriteOptimization(t,
   131  		requestHeader{
   132  			ApiKey:        int16(produce),
   133  			ApiVersion:    int16(v2),
   134  			CorrelationID: testCorrelationID,
   135  			ClientID:      testClientID,
   136  		},
   137  		produceRequestV2{
   138  			RequiredAcks: -1,
   139  			Timeout:      timeout,
   140  			Topics: []produceRequestTopicV2{{
   141  				TopicName: testTopic,
   142  				Partitions: []produceRequestPartitionV2{{
   143  					Partition:      testPartition,
   144  					MessageSetSize: msg.size(), MessageSet: messageSet{msg},
   145  				}},
   146  			}},
   147  		},
   148  		func(w *writeBuffer) {
   149  			w.writeProduceRequestV2(nil, testCorrelationID, testClientID, testTopic, testPartition, timeout*time.Millisecond, -1, Message{
   150  				Offset: 10,
   151  				Key:    key,
   152  				Value:  val,
   153  			})
   154  		},
   155  	)
   156  }
   157  
   158  func testWriteOptimization(t *testing.T, h requestHeader, r request, f func(*writeBuffer)) {
   159  	b1 := &bytes.Buffer{}
   160  	w1 := &writeBuffer{w: b1}
   161  
   162  	b2 := &bytes.Buffer{}
   163  	w2 := &writeBuffer{w: b2}
   164  
   165  	h.Size = (h.size() + r.size()) - 4
   166  	h.writeTo(w1)
   167  	r.writeTo(w1)
   168  
   169  	f(w2)
   170  
   171  	c1 := b1.Bytes()
   172  	c2 := b2.Bytes()
   173  
   174  	if !bytes.Equal(c1, c2) {
   175  		t.Error("content differs")
   176  
   177  		n1 := len(c1)
   178  		n2 := len(c2)
   179  
   180  		if n1 != n2 {
   181  			t.Log("content length 1 =", n1)
   182  			t.Log("content length 2 =", n2)
   183  		} else {
   184  			for i := 0; i != n1; i++ {
   185  				if c1[i] != c2[i] {
   186  					t.Logf("byte at offset %d/%d: %#x != %#x", i, n1, c1[i], c2[i])
   187  					break
   188  				}
   189  			}
   190  		}
   191  	}
   192  }
   193  
   194  func TestWriteV2RecordBatch(t *testing.T) {
   195  	if !ktesting.KafkaIsAtLeast("0.11.0") {
   196  		t.Skip("RecordBatch was added in kafka 0.11.0")
   197  		return
   198  	}
   199  
   200  	client, topic, shutdown := newLocalClientAndTopic()
   201  	defer shutdown()
   202  
   203  	msgs := make([]Message, 15)
   204  	for i := range msgs {
   205  		value := fmt.Sprintf("Sample message content: %d!", i)
   206  		msgs[i] = Message{Key: []byte("Key"), Value: []byte(value), Headers: []Header{{Key: "hk", Value: []byte("hv")}}}
   207  	}
   208  
   209  	w := &Writer{
   210  		Addr:         TCP("localhost:9092"),
   211  		Topic:        topic,
   212  		BatchTimeout: 100 * time.Millisecond,
   213  		BatchSize:    5,
   214  		Transport:    client.Transport,
   215  	}
   216  
   217  	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   218  	defer cancel()
   219  
   220  	if err := w.WriteMessages(ctx, msgs...); err != nil {
   221  		t.Errorf("Failed to write v2 messages to kafka: %v", err)
   222  		return
   223  	}
   224  	w.Close()
   225  
   226  	r := NewReader(ReaderConfig{
   227  		Brokers: []string{"localhost:9092"},
   228  		Topic:   topic,
   229  		MaxWait: 100 * time.Millisecond,
   230  	})
   231  	defer r.Close()
   232  
   233  	msg, err := r.ReadMessage(context.Background())
   234  	if err != nil {
   235  		t.Error("Failed to read message")
   236  		return
   237  	}
   238  
   239  	if string(msg.Key) != "Key" {
   240  		t.Error("Received message's key doesn't match")
   241  		return
   242  	}
   243  	if msg.Headers[0].Key != "hk" {
   244  		t.Error("Received message header's key doesn't match")
   245  		return
   246  	}
   247  }