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