Du kan inte välja fler än 25 ämnen Ämnen måste starta med en bokstav eller siffra, kan innehålla bindestreck ('-') och vara max 35 tecken långa.

443 rader
12 KiB

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