-
Notifications
You must be signed in to change notification settings - Fork 790
/
createtopics.go
390 lines (330 loc) · 11.2 KB
/
createtopics.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
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
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
package kafka
import (
"bufio"
"context"
"fmt"
"net"
"time"
"github.com/segmentio/kafka-go/protocol/createtopics"
)
// CreateTopicRequests represents a request sent to a kafka broker to create
// new topics.
type CreateTopicsRequest struct {
// Address of the kafka broker to send the request to.
Addr net.Addr
// List of topics to create and their configuration.
Topics []TopicConfig
// When set to true, topics are not created but the configuration is
// validated as if they were.
//
// This field will be ignored if the kafka broker did not support the
// CreateTopics API in version 1 or above.
ValidateOnly bool
}
// CreateTopicResponse represents a response from a kafka broker to a topic
// creation request.
type CreateTopicsResponse struct {
// The amount of time that the broker throttled the request.
//
// This field will be zero if the kafka broker did not support the
// CreateTopics API in version 2 or above.
Throttle time.Duration
// Mapping of topic names to errors that occurred while attempting to create
// the topics.
//
// The errors contain the kafka error code. Programs may use the standard
// errors.Is function to test the error against kafka error codes.
Errors map[string]error
}
// CreateTopics sends a topic creation request to a kafka broker and returns the
// response.
func (c *Client) CreateTopics(ctx context.Context, req *CreateTopicsRequest) (*CreateTopicsResponse, error) {
topics := make([]createtopics.RequestTopic, len(req.Topics))
for i, t := range req.Topics {
topics[i] = createtopics.RequestTopic{
Name: t.Topic,
NumPartitions: int32(t.NumPartitions),
ReplicationFactor: int16(t.ReplicationFactor),
Assignments: t.assignments(),
Configs: t.configs(),
}
}
m, err := c.roundTrip(ctx, req.Addr, &createtopics.Request{
Topics: topics,
TimeoutMs: c.timeoutMs(ctx, defaultCreateTopicsTimeout),
ValidateOnly: req.ValidateOnly,
})
if err != nil {
return nil, fmt.Errorf("kafka.(*Client).CreateTopics: %w", err)
}
res := m.(*createtopics.Response)
ret := &CreateTopicsResponse{
Throttle: makeDuration(res.ThrottleTimeMs),
Errors: make(map[string]error, len(res.Topics)),
}
for _, t := range res.Topics {
ret.Errors[t.Name] = makeError(t.ErrorCode, t.ErrorMessage)
}
return ret, nil
}
type ConfigEntry struct {
ConfigName string
ConfigValue string
}
func (c ConfigEntry) toCreateTopicsRequestV0ConfigEntry() createTopicsRequestV0ConfigEntry {
return createTopicsRequestV0ConfigEntry(c)
}
type createTopicsRequestV0ConfigEntry struct {
ConfigName string
ConfigValue string
}
func (t createTopicsRequestV0ConfigEntry) size() int32 {
return sizeofString(t.ConfigName) +
sizeofString(t.ConfigValue)
}
func (t createTopicsRequestV0ConfigEntry) writeTo(wb *writeBuffer) {
wb.writeString(t.ConfigName)
wb.writeString(t.ConfigValue)
}
type ReplicaAssignment struct {
Partition int
// The list of brokers where the partition should be allocated. There must
// be as many entries in thie list as there are replicas of the partition.
// The first entry represents the broker that will be the preferred leader
// for the partition.
//
// This field changed in 0.4 from `int` to `[]int`. It was invalid to pass
// a single integer as this is supposed to be a list. While this introduces
// a breaking change, it probably never worked before.
Replicas []int
}
func (a *ReplicaAssignment) partitionIndex() int32 {
return int32(a.Partition)
}
func (a *ReplicaAssignment) brokerIDs() []int32 {
if len(a.Replicas) == 0 {
return nil
}
replicas := make([]int32, len(a.Replicas))
for i, r := range a.Replicas {
replicas[i] = int32(r)
}
return replicas
}
func (a ReplicaAssignment) toCreateTopicsRequestV0ReplicaAssignment() createTopicsRequestV0ReplicaAssignment {
return createTopicsRequestV0ReplicaAssignment{
Partition: int32(a.Partition),
Replicas: a.brokerIDs(),
}
}
type createTopicsRequestV0ReplicaAssignment struct {
Partition int32
Replicas []int32
}
func (t createTopicsRequestV0ReplicaAssignment) size() int32 {
return sizeofInt32(t.Partition) +
(int32(len(t.Replicas)+1) * sizeofInt32(0)) // N+1 because the array length is a int32
}
func (t createTopicsRequestV0ReplicaAssignment) writeTo(wb *writeBuffer) {
wb.writeInt32(t.Partition)
wb.writeInt32(int32(len(t.Replicas)))
for _, r := range t.Replicas {
wb.writeInt32(int32(r))
}
}
type TopicConfig struct {
// Topic name
Topic string
// NumPartitions created. -1 indicates unset.
NumPartitions int
// ReplicationFactor for the topic. -1 indicates unset.
ReplicationFactor int
// ReplicaAssignments among kafka brokers for this topic partitions. If this
// is set num_partitions and replication_factor must be unset.
ReplicaAssignments []ReplicaAssignment
// ConfigEntries holds topic level configuration for topic to be set.
ConfigEntries []ConfigEntry
}
func (t *TopicConfig) assignments() []createtopics.RequestAssignment {
if len(t.ReplicaAssignments) == 0 {
return nil
}
assignments := make([]createtopics.RequestAssignment, len(t.ReplicaAssignments))
for i, a := range t.ReplicaAssignments {
assignments[i] = createtopics.RequestAssignment{
PartitionIndex: a.partitionIndex(),
BrokerIDs: a.brokerIDs(),
}
}
return assignments
}
func (t *TopicConfig) configs() []createtopics.RequestConfig {
if len(t.ConfigEntries) == 0 {
return nil
}
configs := make([]createtopics.RequestConfig, len(t.ConfigEntries))
for i, c := range t.ConfigEntries {
configs[i] = createtopics.RequestConfig{
Name: c.ConfigName,
Value: c.ConfigValue,
}
}
return configs
}
func (t TopicConfig) toCreateTopicsRequestV0Topic() createTopicsRequestV0Topic {
requestV0ReplicaAssignments := make([]createTopicsRequestV0ReplicaAssignment, 0, len(t.ReplicaAssignments))
for _, a := range t.ReplicaAssignments {
requestV0ReplicaAssignments = append(
requestV0ReplicaAssignments,
a.toCreateTopicsRequestV0ReplicaAssignment())
}
requestV0ConfigEntries := make([]createTopicsRequestV0ConfigEntry, 0, len(t.ConfigEntries))
for _, c := range t.ConfigEntries {
requestV0ConfigEntries = append(
requestV0ConfigEntries,
c.toCreateTopicsRequestV0ConfigEntry())
}
return createTopicsRequestV0Topic{
Topic: t.Topic,
NumPartitions: int32(t.NumPartitions),
ReplicationFactor: int16(t.ReplicationFactor),
ReplicaAssignments: requestV0ReplicaAssignments,
ConfigEntries: requestV0ConfigEntries,
}
}
type createTopicsRequestV0Topic struct {
// Topic name
Topic string
// NumPartitions created. -1 indicates unset.
NumPartitions int32
// ReplicationFactor for the topic. -1 indicates unset.
ReplicationFactor int16
// ReplicaAssignments among kafka brokers for this topic partitions. If this
// is set num_partitions and replication_factor must be unset.
ReplicaAssignments []createTopicsRequestV0ReplicaAssignment
// ConfigEntries holds topic level configuration for topic to be set.
ConfigEntries []createTopicsRequestV0ConfigEntry
}
func (t createTopicsRequestV0Topic) size() int32 {
return sizeofString(t.Topic) +
sizeofInt32(t.NumPartitions) +
sizeofInt16(t.ReplicationFactor) +
sizeofArray(len(t.ReplicaAssignments), func(i int) int32 { return t.ReplicaAssignments[i].size() }) +
sizeofArray(len(t.ConfigEntries), func(i int) int32 { return t.ConfigEntries[i].size() })
}
func (t createTopicsRequestV0Topic) writeTo(wb *writeBuffer) {
wb.writeString(t.Topic)
wb.writeInt32(t.NumPartitions)
wb.writeInt16(t.ReplicationFactor)
wb.writeArray(len(t.ReplicaAssignments), func(i int) { t.ReplicaAssignments[i].writeTo(wb) })
wb.writeArray(len(t.ConfigEntries), func(i int) { t.ConfigEntries[i].writeTo(wb) })
}
// See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics
type createTopicsRequestV0 struct {
// Topics contains n array of single topic creation requests. Can not
// have multiple entries for the same topic.
Topics []createTopicsRequestV0Topic
// Timeout ms to wait for a topic to be completely created on the
// controller node. Values <= 0 will trigger topic creation and return immediately
Timeout int32
}
func (t createTopicsRequestV0) size() int32 {
return sizeofArray(len(t.Topics), func(i int) int32 { return t.Topics[i].size() }) +
sizeofInt32(t.Timeout)
}
func (t createTopicsRequestV0) writeTo(wb *writeBuffer) {
wb.writeArray(len(t.Topics), func(i int) { t.Topics[i].writeTo(wb) })
wb.writeInt32(t.Timeout)
}
type createTopicsResponseV0TopicError struct {
// Topic name
Topic string
// ErrorCode holds response error code
ErrorCode int16
}
func (t createTopicsResponseV0TopicError) size() int32 {
return sizeofString(t.Topic) +
sizeofInt16(t.ErrorCode)
}
func (t createTopicsResponseV0TopicError) writeTo(wb *writeBuffer) {
wb.writeString(t.Topic)
wb.writeInt16(t.ErrorCode)
}
func (t *createTopicsResponseV0TopicError) readFrom(r *bufio.Reader, size int) (remain int, err error) {
if remain, err = readString(r, size, &t.Topic); err != nil {
return
}
if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil {
return
}
return
}
// See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics
type createTopicsResponseV0 struct {
TopicErrors []createTopicsResponseV0TopicError
}
func (t createTopicsResponseV0) size() int32 {
return sizeofArray(len(t.TopicErrors), func(i int) int32 { return t.TopicErrors[i].size() })
}
func (t createTopicsResponseV0) writeTo(wb *writeBuffer) {
wb.writeArray(len(t.TopicErrors), func(i int) { t.TopicErrors[i].writeTo(wb) })
}
func (t *createTopicsResponseV0) readFrom(r *bufio.Reader, size int) (remain int, err error) {
fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) {
var topic createTopicsResponseV0TopicError
if fnRemain, fnErr = (&topic).readFrom(r, size); err != nil {
return
}
t.TopicErrors = append(t.TopicErrors, topic)
return
}
if remain, err = readArrayWith(r, size, fn); err != nil {
return
}
return
}
func (c *Conn) createTopics(request createTopicsRequestV0) (createTopicsResponseV0, error) {
var response createTopicsResponseV0
err := c.writeOperation(
func(deadline time.Time, id int32) error {
if request.Timeout == 0 {
now := time.Now()
deadline = adjustDeadlineForRTT(deadline, now, defaultRTT)
request.Timeout = milliseconds(deadlineToTimeout(deadline, now))
}
return c.writeRequest(createTopics, v0, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return response, err
}
for _, tr := range response.TopicErrors {
if tr.ErrorCode == int16(TopicAlreadyExists) {
continue
}
if tr.ErrorCode != 0 {
return response, Error(tr.ErrorCode)
}
}
return response, nil
}
// CreateTopics creates one topic per provided configuration with idempotent
// operational semantics. In other words, if CreateTopics is invoked with a
// configuration for an existing topic, it will have no effect.
func (c *Conn) CreateTopics(topics ...TopicConfig) error {
requestV0Topics := make([]createTopicsRequestV0Topic, 0, len(topics))
for _, t := range topics {
requestV0Topics = append(
requestV0Topics,
t.toCreateTopicsRequestV0Topic())
}
_, err := c.createTopics(createTopicsRequestV0{
Topics: requestV0Topics,
})
return err
}