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
|
package kafka
import (
"bufio"
"bytes"
"context"
"reflect"
"testing"
"time"
ktesting "github.com/segmentio/kafka-go/testing"
)
func TestOffsetFetchResponseV1(t *testing.T) {
item := offsetFetchResponseV1{
Responses: []offsetFetchResponseV1Response{
{
Topic: "a",
PartitionResponses: []offsetFetchResponseV1PartitionResponse{
{
Partition: 2,
Offset: 3,
Metadata: "b",
ErrorCode: 4,
},
},
},
},
}
b := bytes.NewBuffer(nil)
w := &writeBuffer{w: b}
item.writeTo(w)
var found offsetFetchResponseV1
remain, err := (&found).readFrom(bufio.NewReader(b), b.Len())
if err != nil {
t.Error(err)
t.FailNow()
}
if remain != 0 {
t.Errorf("expected 0 remain, got %v", remain)
t.FailNow()
}
if !reflect.DeepEqual(item, found) {
t.Error("expected item and found to be the same")
t.FailNow()
}
}
func TestOffsetFetchRequestWithNoTopic(t *testing.T) {
if !ktesting.KafkaIsAtLeast("0.10.2.0") {
t.Logf("Test %s is not applicable for kafka versions below 0.10.2.0", t.Name())
t.SkipNow()
}
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
topic1 := makeTopic()
defer deleteTopic(t, topic1)
topic2 := makeTopic()
defer deleteTopic(t, topic2)
consumeGroup := makeGroupID()
numMsgs := 50
defer cancel()
r1 := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic1,
GroupID: consumeGroup,
MinBytes: 1,
MaxBytes: 100,
MaxWait: 100 * time.Millisecond,
})
defer r1.Close()
prepareReader(t, ctx, r1, makeTestSequence(numMsgs)...)
r2 := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic2,
GroupID: consumeGroup,
MinBytes: 1,
MaxBytes: 100,
MaxWait: 100 * time.Millisecond,
})
defer r2.Close()
prepareReader(t, ctx, r2, makeTestSequence(numMsgs)...)
for i := 0; i < numMsgs; i++ {
if _, err := r1.ReadMessage(ctx); err != nil {
t.Fatal(err)
}
}
for i := 0; i < numMsgs; i++ {
if _, err := r2.ReadMessage(ctx); err != nil {
t.Fatal(err)
}
}
client := Client{Addr: TCP("localhost:9092")}
topicOffsets, err := client.OffsetFetch(ctx, &OffsetFetchRequest{GroupID: consumeGroup})
if err != nil {
t.Error(err)
t.FailNow()
}
if len(topicOffsets.Topics) != 2 {
t.Error(err)
t.FailNow()
}
}
func TestOffsetFetchRequestWithOneTopic(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
topic1 := makeTopic()
defer deleteTopic(t, topic1)
topic2 := makeTopic()
defer deleteTopic(t, topic2)
consumeGroup := makeGroupID()
numMsgs := 50
defer cancel()
r1 := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic1,
GroupID: consumeGroup,
MinBytes: 1,
MaxBytes: 100,
MaxWait: 100 * time.Millisecond,
})
defer r1.Close()
prepareReader(t, ctx, r1, makeTestSequence(numMsgs)...)
r2 := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic2,
GroupID: consumeGroup,
MinBytes: 1,
MaxBytes: 100,
MaxWait: 100 * time.Millisecond,
})
defer r2.Close()
prepareReader(t, ctx, r2, makeTestSequence(numMsgs)...)
for i := 0; i < numMsgs; i++ {
if _, err := r1.ReadMessage(ctx); err != nil {
t.Fatal(err)
}
}
for i := 0; i < numMsgs; i++ {
if _, err := r2.ReadMessage(ctx); err != nil {
t.Fatal(err)
}
}
client := Client{Addr: TCP("localhost:9092")}
topicOffsets, err := client.OffsetFetch(ctx, &OffsetFetchRequest{GroupID: consumeGroup, Topics: map[string][]int{
topic1: {0},
}})
if err != nil {
t.Error(err)
t.FailNow()
}
if len(topicOffsets.Topics) != 1 {
t.Error(err)
t.FailNow()
}
}
|