github.com/segmentio/kafka-go@v0.4.48-0.20240318174348-3f6244eb34fd/leavegroup_test.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "bytes" 6 "context" 7 "errors" 8 "reflect" 9 "testing" 10 "time" 11 ) 12 13 func TestClientLeaveGroup(t *testing.T) { 14 // In order to get to a leave group call we need to first 15 // join a group then sync the group. 16 topic := makeTopic() 17 client, shutdown := newLocalClient() 18 client.Timeout = time.Minute 19 // Although at higher api versions ClientID is nullable 20 // for some reason the SyncGroup API call errors 21 // when ClientID is null. 22 // The Java Kafka Consumer generates a ClientID if one is not 23 // present or if the provided ClientID is empty. 24 client.Transport.(*Transport).ClientID = "test-client" 25 defer shutdown() 26 27 err := clientCreateTopic(client, topic, 3) 28 if err != nil { 29 t.Fatal(err) 30 } 31 32 groupID := makeGroupID() 33 34 ctx, cancel := context.WithTimeout(context.Background(), time.Minute) 35 defer cancel() 36 respc, err := waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{ 37 Addr: client.Addr, 38 Key: groupID, 39 KeyType: CoordinatorKeyTypeConsumer, 40 }) 41 if err != nil { 42 t.Fatal(err) 43 } 44 45 if respc.Error != nil { 46 t.Fatal(err) 47 } 48 49 groupInstanceID := "group-instance-id" 50 userData := "user-data" 51 52 var rrGroupBalancer RoundRobinGroupBalancer 53 54 req := &JoinGroupRequest{ 55 GroupID: groupID, 56 GroupInstanceID: groupInstanceID, 57 ProtocolType: "consumer", 58 SessionTimeout: time.Minute, 59 RebalanceTimeout: time.Minute, 60 Protocols: []GroupProtocol{ 61 { 62 Name: rrGroupBalancer.ProtocolName(), 63 Metadata: GroupProtocolSubscription{ 64 Topics: []string{topic}, 65 UserData: []byte(userData), 66 OwnedPartitions: map[string][]int{ 67 topic: {0, 1, 2}, 68 }, 69 }, 70 }, 71 }, 72 } 73 74 var resp *JoinGroupResponse 75 76 for { 77 resp, err = client.JoinGroup(ctx, req) 78 if err != nil { 79 t.Fatal(err) 80 } 81 82 if errors.Is(resp.Error, MemberIDRequired) { 83 req.MemberID = resp.MemberID 84 time.Sleep(time.Second) 85 continue 86 } 87 88 if resp.Error != nil { 89 t.Fatal(resp.Error) 90 } 91 break 92 } 93 94 if resp.MemberID != resp.LeaderID { 95 t.Fatal("expected to be group leader") 96 } 97 98 groupMembers := make([]GroupMember, 0, len(resp.Members)) 99 groupUserDataLookup := make(map[string]GroupMember) 100 for _, member := range resp.Members { 101 gm := GroupMember{ 102 ID: member.ID, 103 Topics: member.Metadata.Topics, 104 UserData: member.Metadata.UserData, 105 } 106 groupMembers = append(groupMembers, gm) 107 groupUserDataLookup[member.ID] = gm 108 } 109 110 metaResp, err := client.Metadata(ctx, &MetadataRequest{ 111 Topics: []string{topic}, 112 }) 113 if err != nil { 114 t.Fatal(err) 115 } 116 117 assignments := rrGroupBalancer.AssignGroups(groupMembers, metaResp.Topics[0].Partitions) 118 119 sgRequest := &SyncGroupRequest{ 120 GroupID: groupID, 121 GenerationID: resp.GenerationID, 122 MemberID: resp.MemberID, 123 GroupInstanceID: groupInstanceID, 124 ProtocolType: "consumer", 125 ProtocolName: rrGroupBalancer.ProtocolName(), 126 } 127 128 for member, assignment := range assignments { 129 sgRequest.Assignments = append(sgRequest.Assignments, SyncGroupRequestAssignment{ 130 MemberID: member, 131 Assignment: GroupProtocolAssignment{ 132 AssignedPartitions: assignment, 133 UserData: groupUserDataLookup[member].UserData, 134 }, 135 }) 136 } 137 sgResp, err := client.SyncGroup(ctx, sgRequest) 138 if err != nil { 139 t.Fatal(err) 140 } 141 142 if sgResp.Error != nil { 143 t.Fatal(sgResp.Error) 144 } 145 146 expectedAssignment := GroupProtocolAssignment{ 147 AssignedPartitions: map[string][]int{ 148 topic: {0, 1, 2}, 149 }, 150 UserData: []byte(userData), 151 } 152 153 if !reflect.DeepEqual(sgResp.Assignment, expectedAssignment) { 154 t.Fatalf("\nexpected assignment to be \n%#v \ngot\n%#v", expectedAssignment, sgResp.Assignment) 155 } 156 157 lgResp, err := client.LeaveGroup(ctx, &LeaveGroupRequest{ 158 GroupID: groupID, 159 Members: []LeaveGroupRequestMember{ 160 { 161 ID: resp.MemberID, 162 GroupInstanceID: groupInstanceID, 163 }, 164 }, 165 }) 166 if err != nil { 167 t.Fatal(err) 168 } 169 170 if lgResp.Error != nil { 171 t.Fatal(err) 172 } 173 174 if len(lgResp.Members) != 1 { 175 t.Fatalf("expected 1 member in response, got %#v", lgResp.Members) 176 } 177 178 member := lgResp.Members[0] 179 180 if member.Error != nil { 181 t.Fatalf("unexpected member error %v", member.Error) 182 } 183 184 if member.GroupInstanceID != groupInstanceID { 185 t.Fatalf("expected group instance id to be %s got %s", groupInstanceID, member.GroupInstanceID) 186 } 187 188 if member.ID != resp.MemberID { 189 t.Fatalf("expected member id to be %s got %s", resp.MemberID, member.ID) 190 } 191 } 192 193 func TestLeaveGroupResponseV0(t *testing.T) { 194 item := leaveGroupResponseV0{ 195 ErrorCode: 2, 196 } 197 198 b := bytes.NewBuffer(nil) 199 w := &writeBuffer{w: b} 200 item.writeTo(w) 201 202 var found leaveGroupResponseV0 203 remain, err := (&found).readFrom(bufio.NewReader(b), b.Len()) 204 if err != nil { 205 t.Error(err) 206 t.FailNow() 207 } 208 if remain != 0 { 209 t.Errorf("expected 0 remain, got %v", remain) 210 t.FailNow() 211 } 212 if !reflect.DeepEqual(item, found) { 213 t.Error("expected item and found to be the same") 214 t.FailNow() 215 } 216 }