forked from IBM/sarama
-
Notifications
You must be signed in to change notification settings - Fork 0
/
functional_test.go
179 lines (152 loc) · 3.95 KB
/
functional_test.go
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
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
package sarama
import (
"fmt"
"net"
"os"
"sync"
"testing"
"time"
)
const (
TestBatchSize = 1000
)
var (
kafkaIsAvailable, kafkaShouldBeAvailable bool
kafkaAddr string
)
func init() {
kafkaAddr = os.Getenv("KAFKA_ADDR")
if kafkaAddr == "" {
kafkaAddr = "localhost:6667"
}
c, err := net.Dial("tcp", kafkaAddr)
if err == nil {
kafkaIsAvailable = true
c.Close()
}
kafkaShouldBeAvailable = os.Getenv("CI") != ""
}
func checkKafkaAvailability(t *testing.T) {
if !kafkaIsAvailable {
if kafkaShouldBeAvailable {
t.Fatalf("Kafka broker is not available on %s. Set KAFKA_ADDR to connect to Kafka on a different location.", kafkaAddr)
} else {
t.Skipf("Kafka broker is not available on %s. Set KAFKA_ADDR to connect to Kafka on a different location.", kafkaAddr)
}
}
}
func TestFuncProducing(t *testing.T) {
config := NewProducerConfig()
testProducingMessages(t, config)
}
func TestFuncProducingGzip(t *testing.T) {
config := NewProducerConfig()
config.Compression = CompressionGZIP
testProducingMessages(t, config)
}
func TestFuncProducingSnappy(t *testing.T) {
config := NewProducerConfig()
config.Compression = CompressionSnappy
testProducingMessages(t, config)
}
func TestFuncProducingNoResponse(t *testing.T) {
config := NewProducerConfig()
config.RequiredAcks = NoResponse
testProducingMessages(t, config)
}
func TestFuncProducingFlushing(t *testing.T) {
config := NewProducerConfig()
config.FlushMsgCount = TestBatchSize / 8
config.FlushFrequency = 250 * time.Millisecond
testProducingMessages(t, config)
}
func TestFuncMultiPartitionProduce(t *testing.T) {
checkKafkaAvailability(t)
client, err := NewClient("functional_test", []string{kafkaAddr}, nil)
if err != nil {
t.Fatal(err)
}
defer safeClose(t, client)
config := NewProducerConfig()
config.FlushFrequency = 50 * time.Millisecond
config.FlushMsgCount = 200
config.ChannelBufferSize = 20
config.AckSuccesses = true
producer, err := NewProducer(client, config)
if err != nil {
t.Fatal(err)
}
var wg sync.WaitGroup
wg.Add(TestBatchSize)
for i := 1; i <= TestBatchSize; i++ {
go func(i int, w *sync.WaitGroup) {
defer w.Done()
msg := &MessageToSend{Topic: "multi_partition", Key: nil, Value: StringEncoder(fmt.Sprintf("hur %d", i))}
producer.Input() <- msg
select {
case ret := <-producer.Errors():
t.Fatal(ret.Err)
case <-producer.Successes():
}
}(i, &wg)
}
wg.Wait()
if err := producer.Close(); err != nil {
t.Error(err)
}
}
func testProducingMessages(t *testing.T, config *ProducerConfig) {
checkKafkaAvailability(t)
client, err := NewClient("functional_test", []string{kafkaAddr}, nil)
if err != nil {
t.Fatal(err)
}
defer safeClose(t, client)
consumerConfig := NewConsumerConfig()
consumerConfig.OffsetMethod = OffsetMethodNewest
consumer, err := NewConsumer(client, "single_partition", 0, "functional_test", consumerConfig)
if err != nil {
t.Fatal(err)
}
defer safeClose(t, consumer)
config.AckSuccesses = true
producer, err := NewProducer(client, config)
if err != nil {
t.Fatal(err)
}
expectedResponses := TestBatchSize
for i := 1; i <= TestBatchSize; {
msg := &MessageToSend{Topic: "single_partition", Key: nil, Value: StringEncoder(fmt.Sprintf("testing %d", i))}
select {
case producer.Input() <- msg:
i++
case ret := <-producer.Errors():
t.Fatal(ret.Err)
case <-producer.Successes():
expectedResponses--
}
}
for expectedResponses > 0 {
select {
case ret := <-producer.Errors():
t.Fatal(ret.Err)
case <-producer.Successes():
expectedResponses--
}
}
err = producer.Close()
if err != nil {
t.Error(err)
}
events := consumer.Events()
for i := 1; i <= TestBatchSize; i++ {
select {
case <-time.After(10 * time.Second):
t.Fatal("Not received any more events in the last 10 seconds.")
case event := <-events:
if string(event.Value) != fmt.Sprintf("testing %d", i) {
t.Fatalf("Unexpected message with index %d: %s", i, event.Value)
}
}
}
}