github.com/rbisecke/kafka-go@v0.4.27/joingroup_test.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "bytes" 6 "reflect" 7 "testing" 8 ) 9 10 func TestSaramaCompatibility(t *testing.T) { 11 var ( 12 // sample data from github.com/Shopify/sarama 13 // 14 // See consumer_group_members_test.go 15 // 16 groupMemberMetadata = []byte{ 17 0, 1, // Version 18 0, 0, 0, 2, // Topic array length 19 0, 3, 'o', 'n', 'e', // Topic one 20 0, 3, 't', 'w', 'o', // Topic two 21 0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata 22 } 23 groupMemberAssignment = []byte{ 24 0, 1, // Version 25 0, 0, 0, 1, // Topic array length 26 0, 3, 'o', 'n', 'e', // Topic one 27 0, 0, 0, 3, // Topic one, partition array length 28 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 4, // 0, 2, 4 29 0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata 30 } 31 ) 32 33 t.Run("verify metadata", func(t *testing.T) { 34 var item groupMetadata 35 remain, err := (&item).readFrom(bufio.NewReader(bytes.NewReader(groupMemberMetadata)), len(groupMemberMetadata)) 36 if err != nil { 37 t.Fatalf("bad err: %v", err) 38 } 39 if remain != 0 { 40 t.Fatalf("expected 0; got %v", remain) 41 } 42 43 if v := item.Version; v != 1 { 44 t.Errorf("expected Version 1; got %v", v) 45 } 46 if v := item.Topics; !reflect.DeepEqual([]string{"one", "two"}, v) { 47 t.Errorf(`expected {"one", "two"}; got %v`, v) 48 } 49 if v := item.UserData; !reflect.DeepEqual([]byte{0x01, 0x02, 0x03}, v) { 50 t.Errorf("expected []byte{0x01, 0x02, 0x03}; got %v", v) 51 } 52 }) 53 54 t.Run("verify assignments", func(t *testing.T) { 55 var item groupAssignment 56 remain, err := (&item).readFrom(bufio.NewReader(bytes.NewReader(groupMemberAssignment)), len(groupMemberAssignment)) 57 if err != nil { 58 t.Fatalf("bad err: %v", err) 59 } 60 if remain != 0 { 61 t.Fatalf("expected 0; got %v", remain) 62 } 63 64 if v := item.Version; v != 1 { 65 t.Errorf("expected Version 1; got %v", v) 66 } 67 if v := item.Topics; !reflect.DeepEqual(map[string][]int32{"one": {0, 2, 4}}, v) { 68 t.Errorf(`expected map[string][]int32{"one": {0, 2, 4}}; got %v`, v) 69 } 70 if v := item.UserData; !reflect.DeepEqual([]byte{0x01, 0x02, 0x03}, v) { 71 t.Errorf("expected []byte{0x01, 0x02, 0x03}; got %v", v) 72 } 73 }) 74 } 75 76 func TestMemberMetadata(t *testing.T) { 77 item := groupMetadata{ 78 Version: 1, 79 Topics: []string{"a", "b"}, 80 UserData: []byte(`blah`), 81 } 82 83 b := bytes.NewBuffer(nil) 84 w := &writeBuffer{w: b} 85 item.writeTo(w) 86 87 var found groupMetadata 88 remain, err := (&found).readFrom(bufio.NewReader(b), b.Len()) 89 if err != nil { 90 t.Error(err) 91 t.FailNow() 92 } 93 if remain != 0 { 94 t.Errorf("expected 0 remain, got %v", remain) 95 t.FailNow() 96 } 97 if !reflect.DeepEqual(item, found) { 98 t.Error("expected item and found to be the same") 99 t.FailNow() 100 } 101 } 102 103 func TestJoinGroupResponseV1(t *testing.T) { 104 item := joinGroupResponseV1{ 105 ErrorCode: 2, 106 GenerationID: 3, 107 GroupProtocol: "a", 108 LeaderID: "b", 109 MemberID: "c", 110 Members: []joinGroupResponseMemberV1{ 111 { 112 MemberID: "d", 113 MemberMetadata: []byte("blah"), 114 }, 115 }, 116 } 117 118 b := bytes.NewBuffer(nil) 119 w := &writeBuffer{w: b} 120 item.writeTo(w) 121 122 var found joinGroupResponseV1 123 remain, err := (&found).readFrom(bufio.NewReader(b), b.Len()) 124 if err != nil { 125 t.Error(err) 126 t.FailNow() 127 } 128 if remain != 0 { 129 t.Errorf("expected 0 remain, got %v", remain) 130 t.FailNow() 131 } 132 if !reflect.DeepEqual(item, found) { 133 t.Error("expected item and found to be the same") 134 t.FailNow() 135 } 136 }