github.com/hack0072008/kafka-go@v1.0.1/createtopics.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "context" 6 "fmt" 7 "net" 8 "time" 9 10 "github.com/hack0072008/kafka-go/protocol/createtopics" 11 ) 12 13 // CreateTopicRequests represents a request sent to a kafka broker to create 14 // new topics. 15 type CreateTopicsRequest struct { 16 // Address of the kafka broker to send the request to. 17 Addr net.Addr 18 19 // List of topics to create and their configuration. 20 Topics []TopicConfig 21 22 // When set to true, topics are not created but the configuration is 23 // validated as if they were. 24 // 25 // This field will be ignored if the kafka broker did no support the 26 // CreateTopics API in version 1 or above. 27 ValidateOnly bool 28 } 29 30 // CreateTopicResponse represents a response from a kafka broker to a topic 31 // creation request. 32 type CreateTopicsResponse struct { 33 // The amount of time that the broker throttled the request. 34 // 35 // This field will be zero if the kafka broker did no support the 36 // CreateTopics API in version 2 or above. 37 Throttle time.Duration 38 39 // Mapping of topic names to errors that occurred while attempting to create 40 // the topics. 41 // 42 // The errors contain the kafka error code. Programs may use the standard 43 // errors.Is function to test the error against kafka error codes. 44 Errors map[string]error 45 } 46 47 // CreateTopics sends a topic creation request to a kafka broker and returns the 48 // response. 49 func (c *Client) CreateTopics(ctx context.Context, req *CreateTopicsRequest) (*CreateTopicsResponse, error) { 50 topics := make([]createtopics.RequestTopic, len(req.Topics)) 51 52 for i, t := range req.Topics { 53 topics[i] = createtopics.RequestTopic{ 54 Name: t.Topic, 55 NumPartitions: int32(t.NumPartitions), 56 ReplicationFactor: int16(t.ReplicationFactor), 57 Assignments: t.assignments(), 58 Configs: t.configs(), 59 } 60 } 61 62 m, err := c.roundTrip(ctx, req.Addr, &createtopics.Request{ 63 Topics: topics, 64 TimeoutMs: c.timeoutMs(ctx, defaultCreateTopicsTimeout), 65 ValidateOnly: req.ValidateOnly, 66 }) 67 68 if err != nil { 69 return nil, fmt.Errorf("kafka.(*Client).CreateTopics: %w", err) 70 } 71 72 res := m.(*createtopics.Response) 73 ret := &CreateTopicsResponse{ 74 Throttle: makeDuration(res.ThrottleTimeMs), 75 Errors: make(map[string]error, len(res.Topics)), 76 } 77 78 for _, t := range res.Topics { 79 ret.Errors[t.Name] = makeError(t.ErrorCode, t.ErrorMessage) 80 } 81 82 return ret, nil 83 } 84 85 type ConfigEntry struct { 86 ConfigName string 87 ConfigValue string 88 } 89 90 func (c ConfigEntry) toCreateTopicsRequestV0ConfigEntry() createTopicsRequestV0ConfigEntry { 91 return createTopicsRequestV0ConfigEntry{ 92 ConfigName: c.ConfigName, 93 ConfigValue: c.ConfigValue, 94 } 95 } 96 97 type createTopicsRequestV0ConfigEntry struct { 98 ConfigName string 99 ConfigValue string 100 } 101 102 func (t createTopicsRequestV0ConfigEntry) size() int32 { 103 return sizeofString(t.ConfigName) + 104 sizeofString(t.ConfigValue) 105 } 106 107 func (t createTopicsRequestV0ConfigEntry) writeTo(wb *writeBuffer) { 108 wb.writeString(t.ConfigName) 109 wb.writeString(t.ConfigValue) 110 } 111 112 type ReplicaAssignment struct { 113 Partition int 114 // The list of brokers where the partition should be allocated. There must 115 // be as many entries in thie list as there are replicas of the partition. 116 // The first entry represents the broker that will be the preferred leader 117 // for the partition. 118 // 119 // This field changed in 0.4 from `int` to `[]int`. It was invalid to pass 120 // a single integer as this is supposed to be a list. While this introduces 121 // a breaking change, it probably never worked before. 122 Replicas []int 123 } 124 125 func (a *ReplicaAssignment) partitionIndex() int32 { 126 return int32(a.Partition) 127 } 128 129 func (a *ReplicaAssignment) brokerIDs() []int32 { 130 if len(a.Replicas) == 0 { 131 return nil 132 } 133 replicas := make([]int32, len(a.Replicas)) 134 for i, r := range a.Replicas { 135 replicas[i] = int32(r) 136 } 137 return replicas 138 } 139 140 func (a ReplicaAssignment) toCreateTopicsRequestV0ReplicaAssignment() createTopicsRequestV0ReplicaAssignment { 141 return createTopicsRequestV0ReplicaAssignment{ 142 Partition: int32(a.Partition), 143 Replicas: a.brokerIDs(), 144 } 145 } 146 147 type createTopicsRequestV0ReplicaAssignment struct { 148 Partition int32 149 Replicas []int32 150 } 151 152 func (t createTopicsRequestV0ReplicaAssignment) size() int32 { 153 return sizeofInt32(t.Partition) + 154 (int32(len(t.Replicas)+1) * sizeofInt32(0)) // N+1 because the array length is a int32 155 } 156 157 func (t createTopicsRequestV0ReplicaAssignment) writeTo(wb *writeBuffer) { 158 wb.writeInt32(t.Partition) 159 wb.writeInt32(int32(len(t.Replicas))) 160 for _, r := range t.Replicas { 161 wb.writeInt32(int32(r)) 162 } 163 } 164 165 type TopicConfig struct { 166 // Topic name 167 Topic string 168 169 // NumPartitions created. -1 indicates unset. 170 NumPartitions int 171 172 // ReplicationFactor for the topic. -1 indicates unset. 173 ReplicationFactor int 174 175 // ReplicaAssignments among kafka brokers for this topic partitions. If this 176 // is set num_partitions and replication_factor must be unset. 177 ReplicaAssignments []ReplicaAssignment 178 179 // ConfigEntries holds topic level configuration for topic to be set. 180 ConfigEntries []ConfigEntry 181 } 182 183 func (t *TopicConfig) assignments() []createtopics.RequestAssignment { 184 if len(t.ReplicaAssignments) == 0 { 185 return nil 186 } 187 assignments := make([]createtopics.RequestAssignment, len(t.ReplicaAssignments)) 188 for i, a := range t.ReplicaAssignments { 189 assignments[i] = createtopics.RequestAssignment{ 190 PartitionIndex: a.partitionIndex(), 191 BrokerIDs: a.brokerIDs(), 192 } 193 } 194 return assignments 195 } 196 197 func (t *TopicConfig) configs() []createtopics.RequestConfig { 198 if len(t.ConfigEntries) == 0 { 199 return nil 200 } 201 configs := make([]createtopics.RequestConfig, len(t.ConfigEntries)) 202 for i, c := range t.ConfigEntries { 203 configs[i] = createtopics.RequestConfig{ 204 Name: c.ConfigName, 205 Value: c.ConfigValue, 206 } 207 } 208 return configs 209 } 210 211 func (t TopicConfig) toCreateTopicsRequestV0Topic() createTopicsRequestV0Topic { 212 var requestV0ReplicaAssignments []createTopicsRequestV0ReplicaAssignment 213 for _, a := range t.ReplicaAssignments { 214 requestV0ReplicaAssignments = append( 215 requestV0ReplicaAssignments, 216 a.toCreateTopicsRequestV0ReplicaAssignment()) 217 } 218 var requestV0ConfigEntries []createTopicsRequestV0ConfigEntry 219 for _, c := range t.ConfigEntries { 220 requestV0ConfigEntries = append( 221 requestV0ConfigEntries, 222 c.toCreateTopicsRequestV0ConfigEntry()) 223 } 224 225 return createTopicsRequestV0Topic{ 226 Topic: t.Topic, 227 NumPartitions: int32(t.NumPartitions), 228 ReplicationFactor: int16(t.ReplicationFactor), 229 ReplicaAssignments: requestV0ReplicaAssignments, 230 ConfigEntries: requestV0ConfigEntries, 231 } 232 } 233 234 type createTopicsRequestV0Topic struct { 235 // Topic name 236 Topic string 237 238 // NumPartitions created. -1 indicates unset. 239 NumPartitions int32 240 241 // ReplicationFactor for the topic. -1 indicates unset. 242 ReplicationFactor int16 243 244 // ReplicaAssignments among kafka brokers for this topic partitions. If this 245 // is set num_partitions and replication_factor must be unset. 246 ReplicaAssignments []createTopicsRequestV0ReplicaAssignment 247 248 // ConfigEntries holds topic level configuration for topic to be set. 249 ConfigEntries []createTopicsRequestV0ConfigEntry 250 } 251 252 func (t createTopicsRequestV0Topic) size() int32 { 253 return sizeofString(t.Topic) + 254 sizeofInt32(t.NumPartitions) + 255 sizeofInt16(t.ReplicationFactor) + 256 sizeofArray(len(t.ReplicaAssignments), func(i int) int32 { return t.ReplicaAssignments[i].size() }) + 257 sizeofArray(len(t.ConfigEntries), func(i int) int32 { return t.ConfigEntries[i].size() }) 258 } 259 260 func (t createTopicsRequestV0Topic) writeTo(wb *writeBuffer) { 261 wb.writeString(t.Topic) 262 wb.writeInt32(t.NumPartitions) 263 wb.writeInt16(t.ReplicationFactor) 264 wb.writeArray(len(t.ReplicaAssignments), func(i int) { t.ReplicaAssignments[i].writeTo(wb) }) 265 wb.writeArray(len(t.ConfigEntries), func(i int) { t.ConfigEntries[i].writeTo(wb) }) 266 } 267 268 // See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics 269 type createTopicsRequestV0 struct { 270 // Topics contains n array of single topic creation requests. Can not 271 // have multiple entries for the same topic. 272 Topics []createTopicsRequestV0Topic 273 274 // Timeout ms to wait for a topic to be completely created on the 275 // controller node. Values <= 0 will trigger topic creation and return immediately 276 Timeout int32 277 } 278 279 func (t createTopicsRequestV0) size() int32 { 280 return sizeofArray(len(t.Topics), func(i int) int32 { return t.Topics[i].size() }) + 281 sizeofInt32(t.Timeout) 282 } 283 284 func (t createTopicsRequestV0) writeTo(wb *writeBuffer) { 285 wb.writeArray(len(t.Topics), func(i int) { t.Topics[i].writeTo(wb) }) 286 wb.writeInt32(t.Timeout) 287 } 288 289 type createTopicsResponseV0TopicError struct { 290 // Topic name 291 Topic string 292 293 // ErrorCode holds response error code 294 ErrorCode int16 295 } 296 297 func (t createTopicsResponseV0TopicError) size() int32 { 298 return sizeofString(t.Topic) + 299 sizeofInt16(t.ErrorCode) 300 } 301 302 func (t createTopicsResponseV0TopicError) writeTo(wb *writeBuffer) { 303 wb.writeString(t.Topic) 304 wb.writeInt16(t.ErrorCode) 305 } 306 307 func (t *createTopicsResponseV0TopicError) readFrom(r *bufio.Reader, size int) (remain int, err error) { 308 if remain, err = readString(r, size, &t.Topic); err != nil { 309 return 310 } 311 if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { 312 return 313 } 314 return 315 } 316 317 // See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics 318 type createTopicsResponseV0 struct { 319 TopicErrors []createTopicsResponseV0TopicError 320 } 321 322 func (t createTopicsResponseV0) size() int32 { 323 return sizeofArray(len(t.TopicErrors), func(i int) int32 { return t.TopicErrors[i].size() }) 324 } 325 326 func (t createTopicsResponseV0) writeTo(wb *writeBuffer) { 327 wb.writeArray(len(t.TopicErrors), func(i int) { t.TopicErrors[i].writeTo(wb) }) 328 } 329 330 func (t *createTopicsResponseV0) readFrom(r *bufio.Reader, size int) (remain int, err error) { 331 fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { 332 var topic createTopicsResponseV0TopicError 333 if fnRemain, fnErr = (&topic).readFrom(r, size); err != nil { 334 return 335 } 336 t.TopicErrors = append(t.TopicErrors, topic) 337 return 338 } 339 if remain, err = readArrayWith(r, size, fn); err != nil { 340 return 341 } 342 343 return 344 } 345 346 func (c *Conn) createTopics(request createTopicsRequestV0) (createTopicsResponseV0, error) { 347 var response createTopicsResponseV0 348 349 err := c.writeOperation( 350 func(deadline time.Time, id int32) error { 351 if request.Timeout == 0 { 352 now := time.Now() 353 deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) 354 request.Timeout = milliseconds(deadlineToTimeout(deadline, now)) 355 } 356 return c.writeRequest(createTopics, v0, id, request) 357 }, 358 func(deadline time.Time, size int) error { 359 return expectZeroSize(func() (remain int, err error) { 360 return (&response).readFrom(&c.rbuf, size) 361 }()) 362 }, 363 ) 364 if err != nil { 365 return response, err 366 } 367 for _, tr := range response.TopicErrors { 368 if tr.ErrorCode != 0 { 369 return response, Error(tr.ErrorCode) 370 } 371 } 372 373 return response, nil 374 } 375 376 // CreateTopics creates one topic per provided configuration with idempotent 377 // operational semantics. In other words, if CreateTopics is invoked with a 378 // configuration for an existing topic, it will have no effect. 379 func (c *Conn) CreateTopics(topics ...TopicConfig) error { 380 var requestV0Topics []createTopicsRequestV0Topic 381 for _, t := range topics { 382 requestV0Topics = append( 383 requestV0Topics, 384 t.toCreateTopicsRequestV0Topic()) 385 } 386 387 _, err := c.createTopics(createTopicsRequestV0{ 388 Topics: requestV0Topics, 389 }) 390 391 switch err { 392 case TopicAlreadyExists: 393 // ok 394 return nil 395 default: 396 return err 397 } 398 }