File: functional_consumer_follower_fetch_test.go

package info (click to toggle)
golang-github-ibm-sarama 1.45.1-1
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 2,964 kB
  • sloc: makefile: 35; sh: 19
file content (126 lines) | stat: -rw-r--r-- 2,582 bytes parent folder | download
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
//go:build functional

package sarama

import (
	"context"
	"fmt"
	"strconv"
	"sync"
	"testing"
	"time"
)

func TestConsumerFetchFollowerFailover(t *testing.T) {
	const (
		topic  = "test.1"
		numMsg = 1000
	)

	newConfig := func() *Config {
		config := NewFunctionalTestConfig()
		config.ClientID = t.Name()
		config.Producer.Return.Successes = true
		return config
	}

	config := newConfig()

	// pick a partition and find the ID for one of the follower brokers
	admin, err := NewClusterAdmin(FunctionalTestEnv.KafkaBrokerAddrs, config)
	if err != nil {
		t.Fatal(err)
	}
	defer admin.Close()

	metadata, err := admin.DescribeTopics([]string{topic})
	if err != nil {
		t.Fatal(err)
	}
	partition := metadata[0].Partitions[0]
	leader := metadata[0].Partitions[0].Leader
	follower := int32(-1)
	for _, replica := range partition.Replicas {
		if replica == leader {
			continue
		}
		follower = replica
		break
	}

	t.Logf("topic %s has leader kafka-%d and our chosen follower is kafka-%d", topic, leader, follower)

	// match our clientID to the given broker so our requests should end up fetching from that follower
	config.RackID = strconv.FormatInt(int64(follower), 10)

	consumer, err := NewConsumer(FunctionalTestEnv.KafkaBrokerAddrs, config)
	if err != nil {
		t.Fatal(err)
	}

	pc, err := consumer.ConsumePartition(topic, partition.ID, OffsetOldest)
	if err != nil {
		t.Fatal(err)
	}
	defer func() {
		pc.Close()
		consumer.Close()
	}()

	producer, err := NewSyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config)
	if err != nil {
		t.Fatal(err)
	}
	defer producer.Close()

	var wg sync.WaitGroup
	wg.Add(numMsg)

	go func() {
		for i := 0; i < numMsg; i++ {
			msg := &ProducerMessage{
				Topic: topic, Key: nil, Value: StringEncoder(fmt.Sprintf("%s %-3d", t.Name(), i)),
			}
			if _, offset, err := producer.SendMessage(msg); err != nil {
				t.Error(i, err)
			} else if offset%50 == 0 {
				t.Logf("sent: %d\n", offset)
			}
			wg.Done()
			time.Sleep(time.Millisecond * 25)
		}
	}()

	i := 0

	for ; i < numMsg/8; i++ {
		msg := <-pc.Messages()
		if msg.Offset%50 == 0 {
			t.Logf("recv: %d\n", msg.Offset)
		}
	}

	if err := stopDockerTestBroker(context.Background(), follower); err != nil {
		t.Fatal(err)
	}

	for ; i < numMsg/3; i++ {
		msg := <-pc.Messages()
		if msg.Offset%50 == 0 {
			t.Logf("recv: %d\n", msg.Offset)
		}
	}

	if err := startDockerTestBroker(context.Background(), follower); err != nil {
		t.Fatal(err)
	}

	for ; i < numMsg; i++ {
		msg := <-pc.Messages()
		if msg.Offset%50 == 0 {
			t.Logf("recv: %d\n", msg.Offset)
		}
	}

	wg.Wait()
}