github.com/streamdal/segmentio-kafka-go@v0.4.47-streamdal/joingroup_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 ktesting "github.com/segmentio/kafka-go/testing" 13 ) 14 15 func TestClientJoinGroup(t *testing.T) { 16 topic := makeTopic() 17 client, shutdown := newLocalClient() 18 defer shutdown() 19 20 err := clientCreateTopic(client, topic, 3) 21 if err != nil { 22 t.Fatal(err) 23 } 24 25 groupID := makeGroupID() 26 27 ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) 28 defer cancel() 29 respc, err := waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{ 30 Addr: client.Addr, 31 Key: groupID, 32 KeyType: CoordinatorKeyTypeConsumer, 33 }) 34 if err != nil { 35 t.Fatal(err) 36 } 37 38 if respc.Error != nil { 39 t.Fatal(err) 40 } 41 42 groupInstanceID := "group-instance-id" 43 if !ktesting.KafkaIsAtLeast("2.4.1") { 44 groupInstanceID = "" 45 } 46 const userData = "user-data" 47 48 req := &JoinGroupRequest{ 49 GroupID: groupID, 50 GroupInstanceID: groupInstanceID, 51 ProtocolType: "consumer", 52 SessionTimeout: time.Minute, 53 RebalanceTimeout: time.Minute, 54 Protocols: []GroupProtocol{ 55 { 56 Name: RoundRobinGroupBalancer{}.ProtocolName(), 57 Metadata: GroupProtocolSubscription{ 58 Topics: []string{topic}, 59 UserData: []byte(userData), 60 OwnedPartitions: map[string][]int{ 61 topic: {0, 1, 2}, 62 }, 63 }, 64 }, 65 }, 66 } 67 68 var resp *JoinGroupResponse 69 70 for { 71 resp, err = client.JoinGroup(ctx, req) 72 if err != nil { 73 t.Fatal(err) 74 } 75 76 if errors.Is(resp.Error, MemberIDRequired) { 77 req.MemberID = resp.MemberID 78 time.Sleep(time.Second) 79 continue 80 } 81 82 if resp.Error != nil { 83 t.Fatal(resp.Error) 84 } 85 break 86 } 87 88 if resp.GenerationID != 1 { 89 t.Fatalf("expected generation ID to be 1 but got %v", resp.GenerationID) 90 } 91 92 if resp.MemberID == "" { 93 t.Fatal("expected a member ID in response") 94 } 95 96 if resp.LeaderID != resp.MemberID { 97 t.Fatalf("expected to be group leader but got %v", resp.LeaderID) 98 } 99 100 if len(resp.Members) != 1 { 101 t.Fatalf("expected 1 member got %v", resp.Members) 102 } 103 104 member := resp.Members[0] 105 106 if member.ID != resp.MemberID { 107 t.Fatal("expected to be the only group memmber") 108 } 109 110 if member.GroupInstanceID != groupInstanceID { 111 t.Fatalf("expected the group instance ID to be %v, got %v", groupInstanceID, member.GroupInstanceID) 112 } 113 114 expectedMetadata := GroupProtocolSubscription{ 115 Topics: []string{topic}, 116 UserData: []byte(userData), 117 OwnedPartitions: map[string][]int{ 118 topic: {0, 1, 2}, 119 }, 120 } 121 122 if !reflect.DeepEqual(member.Metadata, expectedMetadata) { 123 t.Fatalf("\nexpected assignment to be \n%v\nbut got\n%v", expectedMetadata, member.Metadata) 124 } 125 } 126 127 func TestSaramaCompatibility(t *testing.T) { 128 var ( 129 // sample data from github.com/Shopify/sarama 130 // 131 // See consumer_group_members_test.go 132 // 133 groupMemberMetadata = []byte{ 134 0, 1, // Version 135 0, 0, 0, 2, // Topic array length 136 0, 3, 'o', 'n', 'e', // Topic one 137 0, 3, 't', 'w', 'o', // Topic two 138 0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata 139 } 140 groupMemberAssignment = []byte{ 141 0, 1, // Version 142 0, 0, 0, 1, // Topic array length 143 0, 3, 'o', 'n', 'e', // Topic one 144 0, 0, 0, 3, // Topic one, partition array length 145 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 4, // 0, 2, 4 146 0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata 147 } 148 ) 149 150 t.Run("verify metadata", func(t *testing.T) { 151 var item groupMetadata 152 remain, err := (&item).readFrom(bufio.NewReader(bytes.NewReader(groupMemberMetadata)), len(groupMemberMetadata)) 153 if err != nil { 154 t.Fatalf("bad err: %v", err) 155 } 156 if remain != 0 { 157 t.Fatalf("expected 0; got %v", remain) 158 } 159 160 if v := item.Version; v != 1 { 161 t.Errorf("expected Version 1; got %v", v) 162 } 163 if v := item.Topics; !reflect.DeepEqual([]string{"one", "two"}, v) { 164 t.Errorf(`expected {"one", "two"}; got %v`, v) 165 } 166 if v := item.UserData; !reflect.DeepEqual([]byte{0x01, 0x02, 0x03}, v) { 167 t.Errorf("expected []byte{0x01, 0x02, 0x03}; got %v", v) 168 } 169 }) 170 171 t.Run("verify assignments", func(t *testing.T) { 172 var item groupAssignment 173 remain, err := (&item).readFrom(bufio.NewReader(bytes.NewReader(groupMemberAssignment)), len(groupMemberAssignment)) 174 if err != nil { 175 t.Fatalf("bad err: %v", err) 176 } 177 if remain != 0 { 178 t.Fatalf("expected 0; got %v", remain) 179 } 180 181 if v := item.Version; v != 1 { 182 t.Errorf("expected Version 1; got %v", v) 183 } 184 if v := item.Topics; !reflect.DeepEqual(map[string][]int32{"one": {0, 2, 4}}, v) { 185 t.Errorf(`expected map[string][]int32{"one": {0, 2, 4}}; got %v`, v) 186 } 187 if v := item.UserData; !reflect.DeepEqual([]byte{0x01, 0x02, 0x03}, v) { 188 t.Errorf("expected []byte{0x01, 0x02, 0x03}; got %v", v) 189 } 190 }) 191 } 192 193 func TestMemberMetadata(t *testing.T) { 194 item := groupMetadata{ 195 Version: 1, 196 Topics: []string{"a", "b"}, 197 UserData: []byte(`blah`), 198 } 199 200 b := bytes.NewBuffer(nil) 201 w := &writeBuffer{w: b} 202 item.writeTo(w) 203 204 var found groupMetadata 205 remain, err := (&found).readFrom(bufio.NewReader(b), b.Len()) 206 if err != nil { 207 t.Error(err) 208 t.FailNow() 209 } 210 if remain != 0 { 211 t.Errorf("expected 0 remain, got %v", remain) 212 t.FailNow() 213 } 214 if !reflect.DeepEqual(item, found) { 215 t.Error("expected item and found to be the same") 216 t.FailNow() 217 } 218 } 219 220 func TestJoinGroupResponseV1(t *testing.T) { 221 item := joinGroupResponseV1{ 222 ErrorCode: 2, 223 GenerationID: 3, 224 GroupProtocol: "a", 225 LeaderID: "b", 226 MemberID: "c", 227 Members: []joinGroupResponseMemberV1{ 228 { 229 MemberID: "d", 230 MemberMetadata: []byte("blah"), 231 }, 232 }, 233 } 234 235 b := bytes.NewBuffer(nil) 236 w := &writeBuffer{w: b} 237 item.writeTo(w) 238 239 var found joinGroupResponseV1 240 remain, err := (&found).readFrom(bufio.NewReader(b), b.Len()) 241 if err != nil { 242 t.Error(err) 243 t.FailNow() 244 } 245 if remain != 0 { 246 t.Errorf("expected 0 remain, got %v", remain) 247 t.FailNow() 248 } 249 if !reflect.DeepEqual(item, found) { 250 t.Error("expected item and found to be the same") 251 t.FailNow() 252 } 253 }