-
Notifications
You must be signed in to change notification settings - Fork 792
/
builder_test.go
267 lines (244 loc) · 8.13 KB
/
builder_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
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
package kafka
import (
"bytes"
"fmt"
"io"
"time"
"github.com/segmentio/kafka-go/compress"
)
// This file defines builders to assist in creating kafka payloads for unit testing.
// fetchResponseBuilder builds v10 fetch responses. The version of the v10 fetch
// responses are not as important as the message sets contained within, as this
// type is ultimately used to unit test the message set reader that consumes the
// rest of the response once the header has been parsed.
type fetchResponseBuilder struct {
header fetchResponseHeader
msgSets []messageSetBuilder
rendered []byte
}
type fetchResponseHeader struct {
throttle int32
errorCode int16
sessionID int32
topic string
partition int32
partitionErrorCode int16
highWatermarkOffset int64
lastStableOffset int64
logStartOffset int64
}
func (b *fetchResponseBuilder) messages() (res []Message) {
for _, set := range b.msgSets {
res = append(res, set.messages()...)
}
return
}
func (b *fetchResponseBuilder) bytes() []byte {
if b.rendered == nil {
b.rendered = newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt32(b.header.throttle)
wb.writeInt16(b.header.errorCode)
wb.writeInt32(b.header.sessionID)
wb.writeInt32(1) // num topics
wb.writeString(b.header.topic)
wb.writeInt32(1) // how many partitions
wb.writeInt32(b.header.partition)
wb.writeInt16(b.header.partitionErrorCode)
wb.writeInt64(b.header.highWatermarkOffset)
wb.writeInt64(b.header.lastStableOffset)
wb.writeInt64(b.header.logStartOffset)
wb.writeInt32(-1) // num aborted tx
wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
for _, msgSet := range b.msgSets {
wb.Write(msgSet.bytes())
}
}))
})
}
return b.rendered
}
func (b *fetchResponseBuilder) Len() int {
return len(b.bytes())
}
type messageSetBuilder interface {
bytes() []byte
messages() []Message
}
type v0MessageSetBuilder struct {
msgs []Message
codec CompressionCodec
}
func (f v0MessageSetBuilder) messages() []Message {
return f.msgs
}
func (f v0MessageSetBuilder) bytes() []byte {
bs := newWB().call(func(wb *kafkaWriteBuffer) {
for _, msg := range f.msgs {
bs := newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt64(msg.Offset) // offset
wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt32(-1) // crc, unused
wb.writeInt8(0) // magic
wb.writeInt8(0) // attributes -- zero, no compression for the inner message
wb.writeBytes(msg.Key)
wb.writeBytes(msg.Value)
}))
})
wb.Write(bs)
}
})
if f.codec != nil {
bs = newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt64(f.msgs[0].Offset) // offset
wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
compressed := mustCompress(bs, f.codec)
wb.writeInt32(-1) // crc, unused
wb.writeInt8(0) // magic
wb.writeInt8(f.codec.Code()) // attributes
wb.writeBytes(nil) // key is always nil for compressed
wb.writeBytes(compressed) // the value is the compressed message
}))
})
}
return bs
}
type v1MessageSetBuilder struct {
msgs []Message
codec CompressionCodec
}
func (f v1MessageSetBuilder) messages() []Message {
return f.msgs
}
func (f v1MessageSetBuilder) bytes() []byte {
bs := newWB().call(func(wb *kafkaWriteBuffer) {
for i, msg := range f.msgs {
bs := newWB().call(func(wb *kafkaWriteBuffer) {
if f.codec != nil {
wb.writeInt64(int64(i)) // compressed inner message offsets are relative
} else {
wb.writeInt64(msg.Offset) // offset
}
wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt32(-1) // crc, unused
wb.writeInt8(1) // magic
wb.writeInt8(0) // attributes -- zero, no compression for the inner message
wb.writeInt64(1000 * msg.Time.Unix()) // timestamp
wb.writeBytes(msg.Key)
wb.writeBytes(msg.Value)
}))
})
wb.Write(bs)
}
})
if f.codec != nil {
bs = newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt64(f.msgs[len(f.msgs)-1].Offset) // offset of the wrapper message is the last offset of the inner messages
wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
bs := mustCompress(bs, f.codec)
wb.writeInt32(-1) // crc, unused
wb.writeInt8(1) // magic
wb.writeInt8(f.codec.Code()) // attributes
wb.writeInt64(1000 * f.msgs[0].Time.Unix()) // timestamp
wb.writeBytes(nil) // key is always nil for compressed
wb.writeBytes(bs) // the value is the compressed message
}))
})
}
return bs
}
type v2MessageSetBuilder struct {
msgs []Message
codec CompressionCodec
}
func (f v2MessageSetBuilder) messages() []Message {
return f.msgs
}
func (f v2MessageSetBuilder) bytes() []byte {
attributes := int16(0)
if f.codec != nil {
attributes = int16(f.codec.Code()) // set codec code on attributes
}
return newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt64(f.msgs[0].Offset)
wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt32(0) // leader epoch
wb.writeInt8(2) // magic = 2
wb.writeInt32(0) // crc, unused
wb.writeInt16(attributes) // record set attributes
wb.writeInt32(0) // record set last offset delta
wb.writeInt64(1000 * f.msgs[0].Time.Unix()) // record set first timestamp
wb.writeInt64(1000 * f.msgs[0].Time.Unix()) // record set last timestamp
wb.writeInt64(0) // record set producer id
wb.writeInt16(0) // record set producer epoch
wb.writeInt32(0) // record set base sequence
wb.writeInt32(int32(len(f.msgs))) // record set count
bs := newWB().call(func(wb *kafkaWriteBuffer) {
for i, msg := range f.msgs {
wb.Write(newWB().call(func(wb *kafkaWriteBuffer) {
bs := newWB().call(func(wb *kafkaWriteBuffer) {
wb.writeInt8(0) // record attributes, not used here
wb.writeVarInt(1000 * (time.Now().Unix() - msg.Time.Unix())) // timestamp
wb.writeVarInt(int64(i)) // offset delta
wb.writeVarInt(int64(len(msg.Key))) // key len
wb.Write(msg.Key) // key bytes
wb.writeVarInt(int64(len(msg.Value))) // value len
wb.Write(msg.Value) // value bytes
wb.writeVarInt(int64(len(msg.Headers))) // number of headers
for _, header := range msg.Headers {
wb.writeVarInt(int64(len(header.Key)))
wb.Write([]byte(header.Key))
wb.writeVarInt(int64(len(header.Value)))
wb.Write(header.Value)
}
})
wb.writeVarInt(int64(len(bs)))
wb.Write(bs)
}))
}
})
if f.codec != nil {
bs = mustCompress(bs, f.codec)
}
wb.Write(bs)
}))
})
}
// kafkaWriteBuffer is a write buffer that helps writing fetch responses.
type kafkaWriteBuffer struct {
writeBuffer
buf bytes.Buffer
}
func newWB() *kafkaWriteBuffer {
res := kafkaWriteBuffer{}
res.writeBuffer.w = &res.buf
return &res
}
func (f *kafkaWriteBuffer) Bytes() []byte {
return f.buf.Bytes()
}
// call is a convenience method that allows the kafkaWriteBuffer to be used
// in a functional manner. This is helpful when building
// nested structures, as the return value can be fed into
// other fwWB APIs.
func (f *kafkaWriteBuffer) call(cb func(wb *kafkaWriteBuffer)) []byte {
cb(f)
bs := f.Bytes()
if bs == nil {
bs = []byte{}
}
return bs
}
func mustCompress(bs []byte, codec compress.Codec) (res []byte) {
buf := bytes.Buffer{}
codecWriter := codec.NewWriter(&buf)
_, err := io.Copy(codecWriter, bytes.NewReader(bs))
if err != nil {
panic(fmt.Errorf("compress: %w", err))
}
err = codecWriter.Close()
if err != nil {
panic(fmt.Errorf("close codec writer: %w", err))
}
res = buf.Bytes()
return
}