-
Notifications
You must be signed in to change notification settings - Fork 753
/
createtopics.go
288 lines (240 loc) · 7.91 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
package kafka
import (
"bufio"
"time"
)
type ConfigEntry struct {
ConfigName string
ConfigValue string
}
func (c ConfigEntry) toCreateTopicsRequestV2ConfigEntry() createTopicsRequestV2ConfigEntry {
return createTopicsRequestV2ConfigEntry{
ConfigName: c.ConfigName,
ConfigValue: c.ConfigValue,
}
}
type createTopicsRequestV2ConfigEntry struct {
ConfigName string
ConfigValue string
}
func (t createTopicsRequestV2ConfigEntry) size() int32 {
return sizeofString(t.ConfigName) +
sizeofString(t.ConfigValue)
}
func (t createTopicsRequestV2ConfigEntry) writeTo(w *bufio.Writer) {
writeString(w, t.ConfigName)
writeString(w, t.ConfigValue)
}
type ReplicaAssignment struct {
Partition int
Replicas int
}
func (a ReplicaAssignment) toCreateTopicsRequestV2ReplicaAssignment() createTopicsRequestV2ReplicaAssignment {
return createTopicsRequestV2ReplicaAssignment{
Partition: int32(a.Partition),
Replicas: int32(a.Replicas),
}
}
type createTopicsRequestV2ReplicaAssignment struct {
Partition int32
Replicas int32
}
func (t createTopicsRequestV2ReplicaAssignment) size() int32 {
return sizeofInt32(t.Partition) +
sizeofInt32(t.Replicas)
}
func (t createTopicsRequestV2ReplicaAssignment) writeTo(w *bufio.Writer) {
writeInt32(w, t.Partition)
writeInt32(w, t.Replicas)
}
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) toCreateTopicsRequestV2Topic() createTopicsRequestV2Topic {
var requestV2ReplicaAssignments []createTopicsRequestV2ReplicaAssignment
for _, a := range t.ReplicaAssignments {
requestV2ReplicaAssignments = append(
requestV2ReplicaAssignments,
a.toCreateTopicsRequestV2ReplicaAssignment())
}
var requestV2ConfigEntries []createTopicsRequestV2ConfigEntry
for _, c := range t.ConfigEntries {
requestV2ConfigEntries = append(
requestV2ConfigEntries,
c.toCreateTopicsRequestV2ConfigEntry())
}
return createTopicsRequestV2Topic{
Topic: t.Topic,
NumPartitions: int32(t.NumPartitions),
ReplicationFactor: int16(t.ReplicationFactor),
ReplicaAssignments: requestV2ReplicaAssignments,
ConfigEntries: requestV2ConfigEntries,
}
}
type createTopicsRequestV2Topic 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 []createTopicsRequestV2ReplicaAssignment
// ConfigEntries holds topic level configuration for topic to be set.
ConfigEntries []createTopicsRequestV2ConfigEntry
}
func (t createTopicsRequestV2Topic) 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 createTopicsRequestV2Topic) writeTo(w *bufio.Writer) {
writeString(w, t.Topic)
writeInt32(w, t.NumPartitions)
writeInt16(w, t.ReplicationFactor)
writeArray(w, len(t.ReplicaAssignments), func(i int) { t.ReplicaAssignments[i].writeTo(w) })
writeArray(w, len(t.ConfigEntries), func(i int) { t.ConfigEntries[i].writeTo(w) })
}
// See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics
type createTopicsRequestV2 struct {
// Topics contains n array of single topic creation requests. Can not
// have multiple entries for the same topic.
Topics []createTopicsRequestV2Topic
// 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
// ValidateOnly if true, the request will be validated, but the topic won
// 't be created.
ValidateOnly bool
}
func (t createTopicsRequestV2) size() int32 {
return sizeofArray(len(t.Topics), func(i int) int32 { return t.Topics[i].size() }) +
sizeofInt32(t.Timeout) +
sizeofBool(t.ValidateOnly)
}
func (t createTopicsRequestV2) writeTo(w *bufio.Writer) {
writeArray(w, len(t.Topics), func(i int) { t.Topics[i].writeTo(w) })
writeInt32(w, t.Timeout)
writeBool(w, t.ValidateOnly)
}
type createTopicsResponseV2TopicError struct {
// Topic name
Topic string
// ErrorCode holds response error code
ErrorCode int16
// ErrorMessage holds the response error message
ErrorMessage string
}
func (t createTopicsResponseV2TopicError) size() int32 {
return sizeofString(t.Topic) +
sizeofInt16(t.ErrorCode) +
sizeofString(t.ErrorMessage)
}
func (t createTopicsResponseV2TopicError) writeTo(w *bufio.Writer) {
writeString(w, t.Topic)
writeInt16(w, t.ErrorCode)
writeString(w, t.ErrorMessage)
}
func (t *createTopicsResponseV2TopicError) 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
}
if remain, err = readString(r, remain, &t.ErrorMessage); err != nil {
return
}
return
}
// See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics
type createTopicsResponseV2 struct {
ThrottleTimeMS int32
TopicErrors []createTopicsResponseV2TopicError
}
func (t createTopicsResponseV2) size() int32 {
return sizeofInt32(t.ThrottleTimeMS) +
sizeofArray(len(t.TopicErrors), func(i int) int32 { return t.TopicErrors[i].size() })
}
func (t createTopicsResponseV2) writeTo(w *bufio.Writer) {
writeInt32(w, t.ThrottleTimeMS)
writeArray(w, len(t.TopicErrors), func(i int) { t.TopicErrors[i].writeTo(w) })
}
func (t *createTopicsResponseV2) readFrom(r *bufio.Reader, size int) (remain int, err error) {
if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil {
return
}
fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) {
var topic createTopicsResponseV2TopicError
if fnRemain, fnErr = (&topic).readFrom(r, size); err != nil {
return
}
t.TopicErrors = append(t.TopicErrors, topic)
return
}
if remain, err = readArrayWith(r, remain, fn); err != nil {
return
}
return
}
func (c *Conn) createTopics(request createTopicsRequestV2) (createTopicsResponseV2, error) {
var response createTopicsResponseV2
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(createTopicsRequest, v2, 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 != 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 {
var requestV2Topics []createTopicsRequestV2Topic
for _, t := range topics {
requestV2Topics = append(
requestV2Topics,
t.toCreateTopicsRequestV2Topic())
}
_, err := c.createTopics(createTopicsRequestV2{
Topics: requestV2Topics,
})
switch err {
case TopicAlreadyExists:
// ok
return nil
default:
return err
}
}