github.com/hoveychen/kafka-go@v0.4.42/syncgroup_test.go (about)

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