-
Notifications
You must be signed in to change notification settings - Fork 805
/
Copy pathtxnoffsetcommit_test.go
233 lines (200 loc) · 5.3 KB
/
txnoffsetcommit_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
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
package kafka
import (
"context"
"log"
"os"
"strconv"
"testing"
"time"
ktesting "github.com/segmentio/kafka-go/testing"
)
func TestClientTxnOffsetCommit(t *testing.T) {
if !ktesting.KafkaIsAtLeast("0.11.0") {
t.Skip("Skipping test because kafka version is not high enough.")
}
transactionalID := makeTransactionalID()
topic := makeTopic()
client, shutdown := newLocalClientWithTopic(topic, 1)
defer shutdown()
waitForTopic(context.TODO(), t, topic)
defer deleteTopic(t, topic)
now := time.Now()
const N = 10
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()
respc, err := waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{
Addr: client.Addr,
Key: transactionalID,
KeyType: CoordinatorKeyTypeTransaction,
})
if err != nil {
t.Fatal(err)
}
if respc.Error != nil {
t.Fatal(respc.Error)
}
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
respc, err = waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{
Addr: client.Addr,
Key: transactionalID,
KeyType: CoordinatorKeyTypeConsumer,
})
if err != nil {
t.Fatal(err)
}
if respc.Error != nil {
t.Fatal(respc.Error)
}
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
ipResp, err := client.InitProducerID(ctx, &InitProducerIDRequest{
TransactionalID: transactionalID,
TransactionTimeoutMs: 10000,
})
if err != nil {
t.Fatal(err)
}
if ipResp.Error != nil {
t.Fatal(ipResp.Error)
}
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)
}
defer group.Close()
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
gen, err := group.Next(ctx)
if err != nil {
t.Fatal(err)
}
apresp, err := client.AddPartitionsToTxn(ctx, &AddPartitionsToTxnRequest{
TransactionalID: transactionalID,
ProducerID: ipResp.Producer.ProducerID,
ProducerEpoch: ipResp.Producer.ProducerEpoch,
Topics: map[string][]AddPartitionToTxn{
topic: {
{
Partition: 0,
},
},
},
})
if err != nil {
t.Fatal(err)
}
appartition := apresp.Topics[topic]
if len(appartition) != 1 {
t.Fatalf("unexpected partition count; expected: 1, got: %d", len(appartition))
}
for _, partition := range appartition {
if partition.Error != nil {
t.Fatal(partition.Error)
}
}
client.AddOffsetsToTxn(ctx, &AddOffsetsToTxnRequest{
TransactionalID: transactionalID,
ProducerID: ipResp.Producer.ProducerID,
ProducerEpoch: ipResp.Producer.ProducerEpoch,
GroupID: groupID,
})
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
resp, err := client.TxnOffsetCommit(ctx, &TxnOffsetCommitRequest{
TransactionalID: transactionalID,
GroupID: groupID,
MemberID: gen.MemberID,
ProducerID: ipResp.Producer.ProducerID,
ProducerEpoch: ipResp.Producer.ProducerEpoch,
GenerationID: int(gen.ID),
GroupInstanceID: groupID,
Topics: map[string][]TxnOffsetCommit{
topic: {
{
Partition: 0,
Offset: 10,
},
},
},
})
if err != nil {
t.Fatal(err)
}
partitions := resp.Topics[topic]
if len(partitions) != 1 {
t.Fatalf("unexpected partition count; expected: 1, got: %d", len(partitions))
}
for _, partition := range partitions {
if partition.Error != nil {
t.Fatal(partition.Error)
}
}
err = clientEndTxn(client, &EndTxnRequest{
TransactionalID: transactionalID,
ProducerID: ipResp.Producer.ProducerID,
ProducerEpoch: ipResp.Producer.ProducerEpoch,
Committed: true,
})
if err != nil {
t.Fatal(err)
}
// seems like external visibility of the commit isn't
// synchronous with the EndTxn request. This seems
// to give enough time for the commit to become consistently visible.
<-time.After(time.Second)
ofr, err := client.OffsetFetch(ctx, &OffsetFetchRequest{
GroupID: groupID,
Topics: map[string][]int{topic: {0}},
})
if err != nil {
t.Fatal(err)
}
if ofr.Error != nil {
t.Error(ofr.Error)
}
fetresps := ofr.Topics[topic]
if len(fetresps) != 1 {
t.Fatalf("unexpected 1 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)
}
}
}