File: transport_test.go

package info (click to toggle)
golang-github-segmentio-kafka-go 0.4.49%2Bds1-1
  • links: PTS, VCS
  • area: main
  • in suites: sid
  • size: 2,292 kB
  • sloc: sh: 17; makefile: 10
file content (306 lines) | stat: -rw-r--r-- 7,810 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
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
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
package kafka

import (
	"context"
	"crypto/tls"
	"errors"
	"net"
	"testing"
	"time"

	"github.com/segmentio/kafka-go/protocol"
	"github.com/segmentio/kafka-go/protocol/createtopics"
	meta "github.com/segmentio/kafka-go/protocol/metadata"
)

func TestIssue477(t *testing.T) {
	// This test verifies that a connection attempt with a minimal TLS
	// configuration does not panic.
	l, err := net.Listen("tcp", "127.0.0.1:0")
	if err != nil {
		t.Fatal(err)
	}
	defer l.Close()

	cg := connGroup{
		addr: l.Addr(),
		pool: &connPool{
			dial: defaultDialer.DialContext,
			tls:  &tls.Config{},
		},
	}

	if _, err := cg.connect(context.Background(), cg.addr); err != nil {
		// An error is expected here because we are not actually establishing
		// a TLS connection to a kafka broker.
		t.Log(err)
	} else {
		t.Error("no error was reported when attempting to establish a TLS connection to a non-TLS endpoint")
	}
}

func TestIssue672(t *testing.T) {
	// ensure the test times out if the bug is re-introduced
	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
	defer cancel()

	// we'll simulate a situation with one good topic and one bad topic (bad configuration)
	const brokenTopicName = "bad-topic"
	const okTopicName = "good-topic"

	// make the connection pool think it's immediately ready to send
	ready := make(chan struct{})
	close(ready)

	// allow the system to wake as much as it wants
	wake := make(chan event)
	defer close(wake)
	go func() {
		for {
			select {
			case <-ctx.Done():
				return
			case e := <-wake:
				if e == nil {
					return
				}
				e.trigger()
			}
		}
	}()

	// handle requests by immediately resolving them with a create topics response,
	// the "bad topic" will have an error value
	requests := make(chan connRequest, 1)
	defer close(requests)
	go func() {
		request := <-requests
		request.res.resolve(&createtopics.Response{
			ThrottleTimeMs: 0,
			Topics: []createtopics.ResponseTopic{
				{
					Name:         brokenTopicName,
					ErrorCode:    int16(InvalidPartitionNumber),
					ErrorMessage: InvalidPartitionNumber.Description(),
				},
				{
					Name:              okTopicName,
					NumPartitions:     1,
					ReplicationFactor: 1,
				},
			},
		})
	}()

	pool := &connPool{
		ready: ready,
		wake:  wake,
		conns: map[int32]*connGroup{},
	}

	// configure the state so it can find the good topic, but not the one that fails to create
	pool.setState(connPoolState{
		layout: protocol.Cluster{
			Topics: map[string]protocol.Topic{
				okTopicName: {
					Name: okTopicName,
					Partitions: map[int32]protocol.Partition{
						0: {},
					},
				},
			},
		},
	})

	// trick the connection pool into thinking it has a valid connection to a broker
	pool.conns[0] = &connGroup{
		pool:   pool,
		broker: Broker{},
		idleConns: []*conn{
			{
				reqs: requests,
			},
		},
	}

	// perform the round trip:
	// - if the issue is presenting this will hang waiting for metadata to arrive that will
	//   never arrive, causing a deadline timeout.
	// - if the issue is fixed this will resolve almost instantaneously
	r, err := pool.roundTrip(ctx, &createtopics.Request{
		Topics: []createtopics.RequestTopic{
			{
				Name:              brokenTopicName,
				NumPartitions:     0,
				ReplicationFactor: 1,
			},
			{
				Name:              okTopicName,
				NumPartitions:     1,
				ReplicationFactor: 1,
			},
		},
	})
	// detect if the issue is presenting using the context timeout (note that checking the err return value
	// isn't good enough as the original implementation didn't return the context cancellation error due to
	// being run in a defer)
	if errors.Is(ctx.Err(), context.DeadlineExceeded) {
		t.Fatalf("issue 672 is presenting! roundTrip should not have timed out")
	}

	// ancillary assertions as general house-keeping, not directly related to the issue:

	// we're not expecting any errors in this test
	if err != nil {
		t.Fatalf("unexpected error provoking connection pool roundTrip: %v", err)
	}

	// we expect a response containing the errors from the broker
	if r == nil {
		t.Fatal("expected a non-nil response")
	}

	// we expect to have the create topic response with created earlier
	_, ok := r.(*createtopics.Response)
	if !ok {
		t.Fatalf("expected a createtopics.Response but got %T", r)
	}
}

func TestIssue806(t *testing.T) {
	// ensure the test times out if the bug is re-introduced
	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
	defer cancel()

	// simulate unknown topic want auto create with unknownTopicName,
	const unknownTopicName = "unknown-topic"
	const okTopicName = "good-topic"

	// make the connection pool think it's immediately ready to send
	ready := make(chan struct{})
	close(ready)

	// allow the system to wake as much as it wants
	wake := make(chan event)
	defer close(wake)
	go func() {
		for {
			select {
			case <-ctx.Done():
				return
			case e := <-wake:
				if e == nil {
					return
				}
				e.trigger()
			}
		}
	}()

	// handle requests by immediately resolving them with a create topics response,
	// the "unknown topic" will have err UNKNOWN_TOPIC_OR_PARTITION
	requests := make(chan connRequest, 1)
	defer close(requests)
	go func() {
		request := <-requests
		request.res.resolve(&meta.Response{
			Topics: []meta.ResponseTopic{
				{
					Name:      unknownTopicName,
					ErrorCode: int16(UnknownTopicOrPartition),
				},
				{
					Name: okTopicName,
					Partitions: []meta.ResponsePartition{
						{
							PartitionIndex: 0,
						},
					},
				},
			},
		})
	}()

	pool := &connPool{
		ready: ready,
		wake:  wake,
		conns: map[int32]*connGroup{},
	}

	// configure the state,
	//
	// set cached metadata only have good topic,
	// so it need to request metadata,
	// caused by unknown topic cannot find in cached metadata
	//
	// set layout only have good topic,
	// so it can find the good topic, but not the one that fails to create
	pool.setState(connPoolState{
		metadata: &meta.Response{
			Topics: []meta.ResponseTopic{
				{
					Name: okTopicName,
					Partitions: []meta.ResponsePartition{
						{
							PartitionIndex: 0,
						},
					},
				},
			},
		},
		layout: protocol.Cluster{
			Topics: map[string]protocol.Topic{
				okTopicName: {
					Name: okTopicName,
					Partitions: map[int32]protocol.Partition{
						0: {},
					},
				},
			},
		},
	})

	// trick the connection pool into thinking it has a valid connection to request metadata
	pool.ctrl = &connGroup{
		pool:   pool,
		broker: Broker{},
		idleConns: []*conn{
			{
				reqs: requests,
			},
		},
	}

	// perform the round trip:
	// - if the issue is presenting this will hang waiting for metadata to arrive that will
	//   never arrive, causing a deadline timeout.
	// - if the issue is fixed this will resolve almost instantaneously
	r, err := pool.roundTrip(ctx, &meta.Request{
		TopicNames:             []string{unknownTopicName},
		AllowAutoTopicCreation: true,
	})
	// detect if the issue is presenting using the context timeout (note that checking the err return value
	// isn't good enough as the original implementation didn't return the context cancellation error due to
	// being run in a defer)
	if errors.Is(ctx.Err(), context.DeadlineExceeded) {
		t.Fatalf("issue 806 is presenting! roundTrip should not have timed out")
	}

	// ancillary assertions as general house-keeping, not directly related to the issue:

	// we're not expecting any errors in this test
	if err != nil {
		t.Fatalf("unexpected error provoking connection pool roundTrip: %v", err)
	}

	// we expect a response containing the errors from the broker
	if r == nil {
		t.Fatal("expected a non-nil response")
	}

	// we expect to have the create topic response with created earlier
	_, ok := r.(*meta.Response)
	if !ok {
		t.Fatalf("expected a meta.Response but got %T", r)
	}
}