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 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285
|
package kafka
import (
"context"
"errors"
"io"
"io/ioutil"
"net"
"reflect"
"testing"
"time"
"github.com/segmentio/kafka-go/compress"
)
func produceRecords(t *testing.T, n int, addr net.Addr, topic string, compression compress.Codec) []Record {
conn, err := (&Dialer{
Resolver: &net.Resolver{},
}).DialLeader(context.Background(), addr.Network(), addr.String(), topic, 0)
if err != nil {
t.Fatal("failed to open a new kafka connection:", err)
}
defer conn.Close()
msgs := makeTestSequence(n)
if compression == nil {
_, err = conn.WriteMessages(msgs...)
} else {
_, err = conn.WriteCompressedMessages(compression, msgs...)
}
if err != nil {
t.Fatal(err)
}
records := make([]Record, len(msgs))
for offset, msg := range msgs {
records[offset] = Record{
Offset: int64(offset),
Key: NewBytes(msg.Key),
Value: NewBytes(msg.Value),
Headers: msg.Headers,
}
}
return records
}
func TestClientFetch(t *testing.T) {
client, topic, shutdown := newLocalClientAndTopic()
defer shutdown()
records := produceRecords(t, 10, client.Addr, topic, nil)
res, err := client.Fetch(context.Background(), &FetchRequest{
Topic: topic,
Partition: 0,
Offset: 0,
MinBytes: 1,
MaxBytes: 64 * 1024,
MaxWait: 100 * time.Millisecond,
})
if err != nil {
t.Fatal(err)
}
assertFetchResponse(t, res, &FetchResponse{
Topic: topic,
Partition: 0,
HighWatermark: 10,
Records: NewRecordReader(records...),
})
}
func TestClientFetchCompressed(t *testing.T) {
client, topic, shutdown := newLocalClientAndTopic()
defer shutdown()
records := produceRecords(t, 10, client.Addr, topic, &compress.GzipCodec)
res, err := client.Fetch(context.Background(), &FetchRequest{
Topic: topic,
Partition: 0,
Offset: 0,
MinBytes: 1,
MaxBytes: 64 * 1024,
MaxWait: 100 * time.Millisecond,
})
if err != nil {
t.Fatal(err)
}
assertFetchResponse(t, res, &FetchResponse{
Topic: topic,
Partition: 0,
HighWatermark: 10,
Records: NewRecordReader(records...),
})
}
func assertFetchResponse(t *testing.T, found, expected *FetchResponse) {
t.Helper()
if found.Topic != expected.Topic {
t.Error("invalid topic found in response:", found.Topic)
}
if found.Partition != expected.Partition {
t.Error("invalid partition found in response:", found.Partition)
}
if found.HighWatermark != expected.HighWatermark {
t.Error("invalid high watermark found in response:", found.HighWatermark)
}
if found.Error != nil {
t.Error("unexpected error found in response:", found.Error)
}
records1, err := readRecords(found.Records)
if err != nil {
t.Error("error reading records:", err)
}
records2, err := readRecords(expected.Records)
if err != nil {
t.Error("error reading records:", err)
}
assertRecords(t, records1, records2)
}
type memoryRecord struct {
offset int64
key []byte
value []byte
headers []Header
}
func assertRecords(t *testing.T, found, expected []memoryRecord) {
t.Helper()
i := 0
for i < len(found) && i < len(expected) {
r1 := found[i]
r2 := expected[i]
if !reflect.DeepEqual(r1, r2) {
t.Errorf("records at index %d don't match", i)
t.Logf("expected:\n%#v", r2)
t.Logf("found:\n%#v", r1)
}
i++
}
for i < len(found) {
t.Errorf("unexpected record at index %d:\n%+v", i, found[i])
i++
}
for i < len(expected) {
t.Errorf("missing record at index %d:\n%+v", i, expected[i])
i++
}
}
func readRecords(records RecordReader) ([]memoryRecord, error) {
list := []memoryRecord{}
for {
rec, err := records.ReadRecord()
if err != nil {
if errors.Is(err, io.EOF) {
return list, nil
}
return nil, err
}
var (
offset = rec.Offset
key = rec.Key
value = rec.Value
headers = rec.Headers
bytesKey []byte
bytesValues []byte
)
if key != nil {
bytesKey, _ = ioutil.ReadAll(key)
}
if value != nil {
bytesValues, _ = ioutil.ReadAll(value)
}
list = append(list, memoryRecord{
offset: offset,
key: bytesKey,
value: bytesValues,
headers: headers,
})
}
}
func TestClientPipeline(t *testing.T) {
client, topic, shutdown := newLocalClientAndTopic()
defer shutdown()
const numBatches = 100
const recordsPerBatch = 30
unixEpoch := time.Unix(0, 0)
records := make([]Record, recordsPerBatch)
content := []byte("1234567890")
for i := 0; i < numBatches; i++ {
for j := range records {
records[j] = Record{Value: NewBytes(content)}
}
_, err := client.Produce(context.Background(), &ProduceRequest{
Topic: topic,
RequiredAcks: -1,
Records: NewRecordReader(records...),
Compression: Snappy,
})
if err != nil {
t.Fatal(err)
}
}
offset := int64(0)
for i := 0; i < (numBatches * recordsPerBatch); {
req := &FetchRequest{
Topic: topic,
Offset: offset,
MinBytes: 1,
MaxBytes: 8192,
MaxWait: 500 * time.Millisecond,
}
res, err := client.Fetch(context.Background(), req)
if err != nil {
t.Fatal(err)
}
if res.Error != nil {
t.Fatal(res.Error)
}
for {
r, err := res.Records.ReadRecord()
if err != nil {
if errors.Is(err, io.EOF) {
break
}
t.Fatal(err)
}
if r.Key != nil {
r.Key.Close()
}
if r.Value != nil {
r.Value.Close()
}
if r.Offset != offset {
t.Errorf("record at index %d has mismatching offset, want %d but got %d", i, offset, r.Offset)
}
if r.Time.IsZero() || r.Time.Equal(unixEpoch) {
t.Errorf("record at index %d with offset %d has not timestamp", i, r.Offset)
}
offset = r.Offset + 1
i++
}
}
}
|