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