Skip to content

Commit 8ceaf94

Browse files
authored
offsetfetch request topics are now nullable (#1162)
* offsetfetch request topics are now nullable * new unit tests for offsetfetch was added
1 parent c293a8c commit 8ceaf94

File tree

3 files changed

+142
-11
lines changed

3 files changed

+142
-11
lines changed

offsetfetch.go

+19-10
Original file line numberDiff line numberDiff line change
@@ -66,19 +66,28 @@ type OffsetFetchPartition struct {
6666
// OffsetFetch sends an offset fetch request to a kafka broker and returns the
6767
// response.
6868
func (c *Client) OffsetFetch(ctx context.Context, req *OffsetFetchRequest) (*OffsetFetchResponse, error) {
69-
topics := make([]offsetfetch.RequestTopic, 0, len(req.Topics))
7069

71-
for topicName, partitions := range req.Topics {
72-
indexes := make([]int32, len(partitions))
70+
// Kafka version 0.10.2.x and above allow null Topics map for OffsetFetch API
71+
// which will return the result for all topics with the desired consumer group:
72+
// https://kafka.apache.org/0102/protocol.html#The_Messages_OffsetFetch
73+
// For Kafka version below 0.10.2.x this call will result in an error
74+
var topics []offsetfetch.RequestTopic
7375

74-
for i, p := range partitions {
75-
indexes[i] = int32(p)
76-
}
76+
if len(req.Topics) > 0 {
77+
topics = make([]offsetfetch.RequestTopic, 0, len(req.Topics))
78+
79+
for topicName, partitions := range req.Topics {
80+
indexes := make([]int32, len(partitions))
7781

78-
topics = append(topics, offsetfetch.RequestTopic{
79-
Name: topicName,
80-
PartitionIndexes: indexes,
81-
})
82+
for i, p := range partitions {
83+
indexes[i] = int32(p)
84+
}
85+
86+
topics = append(topics, offsetfetch.RequestTopic{
87+
Name: topicName,
88+
PartitionIndexes: indexes,
89+
})
90+
}
8291
}
8392

8493
m, err := c.roundTrip(ctx, req.Addr, &offsetfetch.Request{

offsetfetch_test.go

+122
Original file line numberDiff line numberDiff line change
@@ -3,8 +3,12 @@ package kafka
33
import (
44
"bufio"
55
"bytes"
6+
"context"
67
"reflect"
78
"testing"
9+
"time"
10+
11+
ktesting "github.com/segmentio/kafka-go/testing"
812
)
913

1014
func TestOffsetFetchResponseV1(t *testing.T) {
@@ -43,3 +47,121 @@ func TestOffsetFetchResponseV1(t *testing.T) {
4347
t.FailNow()
4448
}
4549
}
50+
51+
func TestOffsetFetchRequestWithNoTopic(t *testing.T) {
52+
if !ktesting.KafkaIsAtLeast("0.10.2.0") {
53+
t.Logf("Test %s is not applicable for kafka versions below 0.10.2.0", t.Name())
54+
t.SkipNow()
55+
}
56+
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
57+
topic1 := makeTopic()
58+
defer deleteTopic(t, topic1)
59+
topic2 := makeTopic()
60+
defer deleteTopic(t, topic2)
61+
consumeGroup := makeGroupID()
62+
numMsgs := 50
63+
defer cancel()
64+
r1 := NewReader(ReaderConfig{
65+
Brokers: []string{"localhost:9092"},
66+
Topic: topic1,
67+
GroupID: consumeGroup,
68+
MinBytes: 1,
69+
MaxBytes: 100,
70+
MaxWait: 100 * time.Millisecond,
71+
})
72+
defer r1.Close()
73+
prepareReader(t, ctx, r1, makeTestSequence(numMsgs)...)
74+
r2 := NewReader(ReaderConfig{
75+
Brokers: []string{"localhost:9092"},
76+
Topic: topic2,
77+
GroupID: consumeGroup,
78+
MinBytes: 1,
79+
MaxBytes: 100,
80+
MaxWait: 100 * time.Millisecond,
81+
})
82+
defer r2.Close()
83+
prepareReader(t, ctx, r2, makeTestSequence(numMsgs)...)
84+
85+
for i := 0; i < numMsgs; i++ {
86+
if _, err := r1.ReadMessage(ctx); err != nil {
87+
t.Fatal(err)
88+
}
89+
}
90+
for i := 0; i < numMsgs; i++ {
91+
if _, err := r2.ReadMessage(ctx); err != nil {
92+
t.Fatal(err)
93+
}
94+
}
95+
96+
client := Client{Addr: TCP("localhost:9092")}
97+
98+
topicOffsets, err := client.OffsetFetch(ctx, &OffsetFetchRequest{GroupID: consumeGroup})
99+
100+
if err != nil {
101+
t.Error(err)
102+
t.FailNow()
103+
}
104+
105+
if len(topicOffsets.Topics) != 2 {
106+
t.Error(err)
107+
t.FailNow()
108+
}
109+
110+
}
111+
112+
func TestOffsetFetchRequestWithOneTopic(t *testing.T) {
113+
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
114+
topic1 := makeTopic()
115+
defer deleteTopic(t, topic1)
116+
topic2 := makeTopic()
117+
defer deleteTopic(t, topic2)
118+
consumeGroup := makeGroupID()
119+
numMsgs := 50
120+
defer cancel()
121+
r1 := NewReader(ReaderConfig{
122+
Brokers: []string{"localhost:9092"},
123+
Topic: topic1,
124+
GroupID: consumeGroup,
125+
MinBytes: 1,
126+
MaxBytes: 100,
127+
MaxWait: 100 * time.Millisecond,
128+
})
129+
defer r1.Close()
130+
prepareReader(t, ctx, r1, makeTestSequence(numMsgs)...)
131+
r2 := NewReader(ReaderConfig{
132+
Brokers: []string{"localhost:9092"},
133+
Topic: topic2,
134+
GroupID: consumeGroup,
135+
MinBytes: 1,
136+
MaxBytes: 100,
137+
MaxWait: 100 * time.Millisecond,
138+
})
139+
defer r2.Close()
140+
prepareReader(t, ctx, r2, makeTestSequence(numMsgs)...)
141+
142+
for i := 0; i < numMsgs; i++ {
143+
if _, err := r1.ReadMessage(ctx); err != nil {
144+
t.Fatal(err)
145+
}
146+
}
147+
for i := 0; i < numMsgs; i++ {
148+
if _, err := r2.ReadMessage(ctx); err != nil {
149+
t.Fatal(err)
150+
}
151+
}
152+
153+
client := Client{Addr: TCP("localhost:9092")}
154+
topicOffsets, err := client.OffsetFetch(ctx, &OffsetFetchRequest{GroupID: consumeGroup, Topics: map[string][]int{
155+
topic1: {0},
156+
}})
157+
158+
if err != nil {
159+
t.Error(err)
160+
t.FailNow()
161+
}
162+
163+
if len(topicOffsets.Topics) != 1 {
164+
t.Error(err)
165+
t.FailNow()
166+
}
167+
}

protocol/offsetfetch/offsetfetch.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@ func init() {
88

99
type Request struct {
1010
GroupID string `kafka:"min=v0,max=v5"`
11-
Topics []RequestTopic `kafka:"min=v0,max=v5"`
11+
Topics []RequestTopic `kafka:"min=v0,max=v5,nullable"`
1212
}
1313

1414
func (r *Request) ApiKey() protocol.ApiKey { return protocol.OffsetFetch }

0 commit comments

Comments
 (0)