@@ -13,7 +13,8 @@ type AlterPartitionReassignmentsRequest struct {
13
13
// Address of the kafka broker to send the request to.
14
14
Addr net.Addr
15
15
16
- // Topic is the name of the topic to alter partitions in.
16
+ // Topic is the name of the topic to alter partitions in. Keep this field empty and use Topic in AlterPartitionReassignmentsRequestAssignment to
17
+ // reassign to multiple topics.
17
18
Topic string
18
19
19
20
// Assignments is the list of partition reassignments to submit to the API.
@@ -26,10 +27,13 @@ type AlterPartitionReassignmentsRequest struct {
26
27
// AlterPartitionReassignmentsRequestAssignment contains the requested reassignments for a single
27
28
// partition.
28
29
type AlterPartitionReassignmentsRequestAssignment struct {
30
+ // Topic is the name of the topic to alter partitions in. If empty, the value of Topic in AlterPartitionReassignmentsRequest is used.
31
+ Topic string
32
+
29
33
// PartitionID is the ID of the partition to make the reassignments in.
30
34
PartitionID int
31
35
32
- // BrokerIDs is a slice of brokers to set the partition replicas to.
36
+ // BrokerIDs is a slice of brokers to set the partition replicas to, or null to cancel a pending reassignment for this partition .
33
37
BrokerIDs []int
34
38
}
35
39
@@ -46,6 +50,9 @@ type AlterPartitionReassignmentsResponse struct {
46
50
// AlterPartitionReassignmentsResponsePartitionResult contains the detailed result of
47
51
// doing reassignments for a single partition.
48
52
type AlterPartitionReassignmentsResponsePartitionResult struct {
53
+ // Topic is the topic name.
54
+ Topic string
55
+
49
56
// PartitionID is the ID of the partition that was altered.
50
57
PartitionID int
51
58
@@ -58,16 +65,29 @@ func (c *Client) AlterPartitionReassignments(
58
65
ctx context.Context ,
59
66
req * AlterPartitionReassignmentsRequest ,
60
67
) (* AlterPartitionReassignmentsResponse , error ) {
61
- apiPartitions := [] alterpartitionreassignments.RequestPartition {}
68
+ apiTopicMap := make ( map [ string ] * alterpartitionreassignments.RequestTopic )
62
69
63
70
for _ , assignment := range req .Assignments {
71
+ topic := assignment .Topic
72
+ if topic == "" {
73
+ topic = req .Topic
74
+ }
75
+
76
+ apiTopic := apiTopicMap [topic ]
77
+ if apiTopic == nil {
78
+ apiTopic = & alterpartitionreassignments.RequestTopic {
79
+ Name : topic ,
80
+ }
81
+ apiTopicMap [topic ] = apiTopic
82
+ }
83
+
64
84
replicas := []int32 {}
65
85
for _ , brokerID := range assignment .BrokerIDs {
66
86
replicas = append (replicas , int32 (brokerID ))
67
87
}
68
88
69
- apiPartitions = append (
70
- apiPartitions ,
89
+ apiTopic . Partitions = append (
90
+ apiTopic . Partitions ,
71
91
alterpartitionreassignments.RequestPartition {
72
92
PartitionIndex : int32 (assignment .PartitionID ),
73
93
Replicas : replicas ,
@@ -77,12 +97,10 @@ func (c *Client) AlterPartitionReassignments(
77
97
78
98
apiReq := & alterpartitionreassignments.Request {
79
99
TimeoutMs : int32 (req .Timeout .Milliseconds ()),
80
- Topics : []alterpartitionreassignments.RequestTopic {
81
- {
82
- Name : req .Topic ,
83
- Partitions : apiPartitions ,
84
- },
85
- },
100
+ }
101
+
102
+ for _ , apiTopic := range apiTopicMap {
103
+ apiReq .Topics = append (apiReq .Topics , * apiTopic )
86
104
}
87
105
88
106
protoResp , err := c .roundTrip (
@@ -104,6 +122,7 @@ func (c *Client) AlterPartitionReassignments(
104
122
resp .PartitionResults = append (
105
123
resp .PartitionResults ,
106
124
AlterPartitionReassignmentsResponsePartitionResult {
125
+ Topic : topicResult .Name ,
107
126
PartitionID : int (partitionResult .PartitionIndex ),
108
127
Error : makeError (partitionResult .ErrorCode , partitionResult .ErrorMessage ),
109
128
},
0 commit comments