github.com/segmentio/kafka-go@v0.4.48-0.20240318174348-3f6244eb34fd/offsetcommit_test.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "bytes" 6 "context" 7 "log" 8 "os" 9 "reflect" 10 "strconv" 11 "testing" 12 "time" 13 ) 14 15 func TestOffsetCommitResponseV2(t *testing.T) { 16 item := offsetCommitResponseV2{ 17 Responses: []offsetCommitResponseV2Response{ 18 { 19 Topic: "a", 20 PartitionResponses: []offsetCommitResponseV2PartitionResponse{ 21 { 22 Partition: 1, 23 ErrorCode: 2, 24 }, 25 }, 26 }, 27 }, 28 } 29 30 b := bytes.NewBuffer(nil) 31 w := &writeBuffer{w: b} 32 item.writeTo(w) 33 34 var found offsetCommitResponseV2 35 remain, err := (&found).readFrom(bufio.NewReader(b), b.Len()) 36 if err != nil { 37 t.Error(err) 38 t.FailNow() 39 } 40 if remain != 0 { 41 t.Errorf("expected 0 remain, got %v", remain) 42 t.FailNow() 43 } 44 if !reflect.DeepEqual(item, found) { 45 t.Error("expected item and found to be the same") 46 t.FailNow() 47 } 48 } 49 50 func TestClientOffsetCommit(t *testing.T) { 51 topic := makeTopic() 52 client, shutdown := newLocalClientWithTopic(topic, 3) 53 defer shutdown() 54 now := time.Now() 55 56 const N = 10 * 3 57 records := make([]Record, 0, N) 58 for i := 0; i < N; i++ { 59 records = append(records, Record{ 60 Time: now, 61 Value: NewBytes([]byte("test-message-" + strconv.Itoa(i))), 62 }) 63 } 64 ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) 65 defer cancel() 66 res, err := client.Produce(ctx, &ProduceRequest{ 67 Topic: topic, 68 RequiredAcks: RequireAll, 69 Records: NewRecordReader(records...), 70 }) 71 if err != nil { 72 t.Fatal(err) 73 } 74 75 if res.Error != nil { 76 t.Error(res.Error) 77 } 78 79 for index, err := range res.RecordErrors { 80 t.Fatalf("record at index %d produced an error: %v", index, err) 81 } 82 ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) 83 defer cancel() 84 groupID := makeGroupID() 85 86 group, err := NewConsumerGroup(ConsumerGroupConfig{ 87 ID: groupID, 88 Topics: []string{topic}, 89 Brokers: []string{"localhost:9092"}, 90 HeartbeatInterval: 2 * time.Second, 91 RebalanceTimeout: 2 * time.Second, 92 RetentionTime: time.Hour, 93 Logger: log.New(os.Stdout, "cg-test: ", 0), 94 }) 95 if err != nil { 96 t.Fatal(err) 97 } 98 defer group.Close() 99 100 gen, err := group.Next(ctx) 101 if err != nil { 102 t.Fatal(err) 103 } 104 105 ocr, err := client.OffsetCommit(ctx, &OffsetCommitRequest{ 106 Addr: nil, 107 GroupID: groupID, 108 GenerationID: int(gen.ID), 109 MemberID: gen.MemberID, 110 Topics: map[string][]OffsetCommit{ 111 topic: { 112 {Partition: 0, Offset: 10}, 113 {Partition: 1, Offset: 10}, 114 {Partition: 2, Offset: 10}, 115 }, 116 }, 117 }) 118 if err != nil { 119 t.Fatal(err) 120 } 121 122 resps := ocr.Topics[topic] 123 if len(resps) != 3 { 124 t.Fatalf("expected 3 offsetcommitpartition responses; got %d", len(resps)) 125 } 126 127 for _, resp := range resps { 128 if resp.Error != nil { 129 t.Fatal(resp.Error) 130 } 131 } 132 133 ofr, err := client.OffsetFetch(ctx, &OffsetFetchRequest{ 134 GroupID: groupID, 135 Topics: map[string][]int{topic: {0, 1, 2}}, 136 }) 137 if err != nil { 138 t.Fatal(err) 139 } 140 141 if ofr.Error != nil { 142 t.Error(res.Error) 143 } 144 145 fetresps := ofr.Topics[topic] 146 if len(fetresps) != 3 { 147 t.Fatalf("expected 3 offsetfetchpartition responses; got %d", len(resps)) 148 } 149 150 for _, r := range fetresps { 151 if r.Error != nil { 152 t.Fatal(r.Error) 153 } 154 155 if r.CommittedOffset != 10 { 156 t.Fatalf("expected committed offset to be 10; got: %v for partition: %v", r.CommittedOffset, r.Partition) 157 } 158 } 159 }