forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
write.go
501 lines (424 loc) · 11.1 KB
/
write.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
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
package kafka
import (
"bufio"
"bytes"
"encoding/binary"
"fmt"
"hash/crc32"
"math"
"time"
)
type writable interface {
writeTo(*bufio.Writer)
}
func writeInt8(w *bufio.Writer, i int8) {
w.WriteByte(byte(i))
}
func writeInt16(w *bufio.Writer, i int16) {
var b [2]byte
binary.BigEndian.PutUint16(b[:], uint16(i))
w.WriteByte(b[0])
w.WriteByte(b[1])
}
func writeInt32(w *bufio.Writer, i int32) {
var b [4]byte
binary.BigEndian.PutUint32(b[:], uint32(i))
w.WriteByte(b[0])
w.WriteByte(b[1])
w.WriteByte(b[2])
w.WriteByte(b[3])
}
func writeInt64(w *bufio.Writer, i int64) {
var b [8]byte
binary.BigEndian.PutUint64(b[:], uint64(i))
w.WriteByte(b[0])
w.WriteByte(b[1])
w.WriteByte(b[2])
w.WriteByte(b[3])
w.WriteByte(b[4])
w.WriteByte(b[5])
w.WriteByte(b[6])
w.WriteByte(b[7])
}
func writeVarInt(w *bufio.Writer, i int64) {
i = i<<1 ^ i>>63
for i&0x7f != i {
w.WriteByte(byte(i&0x7f | 0x80))
i >>= 7
}
w.WriteByte(byte(i))
}
func varIntLen(i int64) (l int) {
i = i<<1 ^ i>>63
for i&0x7f != i {
l++
i >>= 7
}
l++
return l
}
func writeString(w *bufio.Writer, s string) {
writeInt16(w, int16(len(s)))
w.WriteString(s)
}
func writeBytes(w *bufio.Writer, b []byte) {
n := len(b)
if b == nil {
n = -1
}
writeInt32(w, int32(n))
w.Write(b)
}
func writeBool(w *bufio.Writer, b bool) {
v := int8(0)
if b {
v = 1
}
writeInt8(w, v)
}
func writeArrayLen(w *bufio.Writer, n int) {
writeInt32(w, int32(n))
}
func writeArray(w *bufio.Writer, n int, f func(int)) {
writeArrayLen(w, n)
for i := 0; i != n; i++ {
f(i)
}
}
func writeStringArray(w *bufio.Writer, a []string) {
writeArray(w, len(a), func(i int) { writeString(w, a[i]) })
}
func writeInt32Array(w *bufio.Writer, a []int32) {
writeArray(w, len(a), func(i int) { writeInt32(w, a[i]) })
}
func write(w *bufio.Writer, a interface{}) {
switch v := a.(type) {
case int8:
writeInt8(w, v)
case int16:
writeInt16(w, v)
case int32:
writeInt32(w, v)
case int64:
writeInt64(w, v)
case string:
writeString(w, v)
case []byte:
writeBytes(w, v)
case bool:
writeBool(w, v)
case writable:
v.writeTo(w)
default:
panic(fmt.Sprintf("unsupported type: %T", a))
}
}
// The functions bellow are used as optimizations to avoid dynamic memory
// allocations that occur when building the data structures representing the
// kafka protocol requests.
func writeFetchRequestV2(w *bufio.Writer, correlationID int32, clientID, topic string, partition int32, offset int64, minBytes, maxBytes int, maxWait time.Duration) error {
h := requestHeader{
ApiKey: int16(fetchRequest),
ApiVersion: int16(v2),
CorrelationID: correlationID,
ClientID: clientID,
}
h.Size = (h.size() - 4) +
4 + // replica ID
4 + // max wait time
4 + // min bytes
4 + // topic array length
sizeofString(topic) +
4 + // partition array length
4 + // partition
8 + // offset
4 // max bytes
h.writeTo(w)
writeInt32(w, -1) // replica ID
writeInt32(w, milliseconds(maxWait))
writeInt32(w, int32(minBytes))
// topic array
writeArrayLen(w, 1)
writeString(w, topic)
// partition array
writeArrayLen(w, 1)
writeInt32(w, partition)
writeInt64(w, offset)
writeInt32(w, int32(maxBytes))
return w.Flush()
}
func writeListOffsetRequestV1(w *bufio.Writer, correlationID int32, clientID, topic string, partition int32, time int64) error {
h := requestHeader{
ApiKey: int16(listOffsetRequest),
ApiVersion: int16(v1),
CorrelationID: correlationID,
ClientID: clientID,
}
h.Size = (h.size() - 4) +
4 + // replica ID
4 + // topic array length
sizeofString(topic) + // topic
4 + // partition array length
4 + // partition
8 // time
h.writeTo(w)
writeInt32(w, -1) // replica ID
// topic array
writeArrayLen(w, 1)
writeString(w, topic)
// partition array
writeArrayLen(w, 1)
writeInt32(w, partition)
writeInt64(w, time)
return w.Flush()
}
func hasHeaders(msgs ...Message) bool {
for _, msg := range msgs {
if len(msg.Headers) > 0 {
return true
}
}
return false
}
func writeProduceRequestV2(w *bufio.Writer, codec CompressionCodec, correlationID int32, clientID, topic string, partition int32, timeout time.Duration, requiredAcks int16, msgs ...Message) (err error) {
attributes := int8(CompressionNoneCode)
var size int32
if hasHeaders(msgs...) {
size = recordBatchSize(msgs...)
} else {
if codec != nil {
if msgs, err = compress(codec, msgs...); err != nil {
return err
}
attributes = codec.Code()
}
size = messageSetSize(msgs...)
}
h := requestHeader{
ApiKey: int16(produceRequest),
ApiVersion: int16(v2),
CorrelationID: correlationID,
ClientID: clientID,
}
h.Size = (h.size() - 4) +
2 + // required acks
4 + // timeout
4 + // topic array length
sizeofString(topic) + // topic
4 + // partition array length
4 + // partition
4 + // message set size
size
h.writeTo(w)
writeInt16(w, requiredAcks) // required acks
writeInt32(w, milliseconds(timeout))
// topic array
writeArrayLen(w, 1)
writeString(w, topic)
// partition array
writeArrayLen(w, 1)
writeInt32(w, partition)
writeInt32(w, size)
if hasHeaders(msgs...) {
err = writeRecordBatch(w, codec, correlationID, clientID, topic, partition, timeout, requiredAcks, msgs...)
} else {
for _, msg := range msgs {
writeMessage(w, msg.Offset, attributes, msg.Time, msg.Key, msg.Value)
}
}
if err != nil {
return
}
return w.Flush()
}
func messageSetSize(msgs ...Message) (size int32) {
for _, msg := range msgs {
size += 8 + // offset
4 + // message size
4 + // crc
1 + // magic byte
1 + // attributes
8 + // timestamp
sizeofBytes(msg.Key) +
sizeofBytes(msg.Value)
}
return
}
func recordBatchSize(msgs ...Message) (size int32) {
size = 8 + // base offset
4 + // batch length
4 + // partition leader epoch
1 + // magic
4 + // crc
2 + // attributes
4 + // last offset delta
8 + // first timestamp
8 + // max timestamp
8 + // producer id
2 + // producer epoch
4 + // base sequence
4 // msg count
baseTime := msgs[0].Time
baseOffset := baseOffset(msgs...)
for _, msg := range msgs {
sz := recordSize(&msg, msg.Time.Sub(baseTime), msg.Offset-baseOffset)
size += int32(sz + varIntLen(int64(sz)))
}
return
}
func writeRecordBatch(w *bufio.Writer, codec CompressionCodec, correlationID int32, clientId, topic string, partition int32, timeout time.Duration, requiredAcks int16, msgs ...Message) error {
baseTime := msgs[0].Time
baseOffset := int64(0)
for i := 0; i < len(msgs); i++ {
msgs[i].Offset = int64(i)
}
size := recordBatchSize(msgs...)
writeInt64(w, baseOffset)
writeInt32(w, int32(size-12)) // 12 = batch length + base offset sizes
writeInt32(w, -1) // partition leader epoch
writeInt8(w, 2) // magic byte
crcBuf := &bytes.Buffer{}
crcBuf.Grow(int(size - 12)) // 12 = batch length + base offset sizes
crcWriter := bufio.NewWriter(crcBuf)
writeInt16(crcWriter, 0) // attributes no compression, timestamp type 0 - create time, not part of a transaction, no control messages
writeInt32(crcWriter, int32(len(msgs)-1)) // max offset
writeInt64(crcWriter, timestamp(baseTime))
lastTime := timestamp(msgs[len(msgs)-1].Time)
writeInt64(crcWriter, int64(lastTime))
writeInt64(crcWriter, -1) // default producer id for now
writeInt16(crcWriter, -1) // default producer epoch for now
writeInt32(crcWriter, -1) // default base sequence
writeInt32(crcWriter, int32(len(msgs))) // record count
for _, msg := range msgs {
writeRecord(crcWriter, CompressionNoneCode, baseTime, baseOffset, msg)
}
crcWriter.Flush()
crcTable := crc32.MakeTable(crc32.Castagnoli)
crcChecksum := crc32.Checksum(crcBuf.Bytes(), crcTable)
writeInt32(w, int32(crcChecksum))
w.Write(crcBuf.Bytes())
return nil
}
var maxDate time.Time = time.Date(5000, time.January, 0, 0, 0, 0, 0, time.UTC)
func baseTime(msgs ...Message) (baseTime time.Time) {
baseTime = maxDate
for _, msg := range msgs {
if msg.Time.Before(baseTime) {
baseTime = msg.Time
}
}
return
}
func baseOffset(msgs ...Message) (baseOffset int64) {
baseOffset = math.MaxInt64
for _, msg := range msgs {
if msg.Offset < baseOffset {
baseOffset = msg.Offset
}
}
return
}
func maxOffsetDelta(baseOffset int64, msgs ...Message) (maxDelta int64) {
maxDelta = 0
for _, msg := range msgs {
curDelta := msg.Offset - baseOffset
if maxDelta > curDelta {
maxDelta = curDelta
}
}
return
}
func estimatedRecordSize(msg *Message) (size int32) {
size += 8 + // length
1 + // attributes
8 + // timestamp delta
8 + // offset delta
8 + // key length
int32(len(msg.Key)) +
8 + // value length
int32(len(msg.Value))
for _, h := range msg.Headers {
size += 8 + // header key length
int32(len(h.Key)) +
8 + // header value length
int32(len(h.Value))
}
return
}
func recordSize(msg *Message, timestampDelta time.Duration, offsetDelta int64) (size int) {
size += 1 + // attributes
varIntLen(int64(timestampDelta)) +
varIntLen(offsetDelta) +
varIntLen(int64(len(msg.Key))) +
len(msg.Key) +
varIntLen(int64(len(msg.Value))) +
len(msg.Value) +
varIntLen(int64(len(msg.Headers)))
for _, h := range msg.Headers {
size += varIntLen(int64(len([]byte(h.Key)))) +
len([]byte(h.Key)) +
varIntLen(int64(len(h.Value))) +
len(h.Value)
}
return
}
func compress(codec CompressionCodec, msgs ...Message) ([]Message, error) {
estimatedLen := 0
for _, msg := range msgs {
estimatedLen += int(msgSize(msg.Key, msg.Value))
}
buf := &bytes.Buffer{}
buf.Grow(estimatedLen)
bufWriter := bufio.NewWriter(buf)
for offset, msg := range msgs {
writeMessage(bufWriter, int64(offset), CompressionNoneCode, msg.Time, msg.Key, msg.Value)
}
bufWriter.Flush()
compressed, err := codec.Encode(buf.Bytes())
if err != nil {
return nil, err
}
return []Message{{Value: compressed}}, nil
}
const magicByte = 1 // compatible with kafka 0.10.0.0+
func writeMessage(w *bufio.Writer, offset int64, attributes int8, time time.Time, key, value []byte) {
timestamp := timestamp(time)
crc32 := crc32OfMessage(magicByte, attributes, timestamp, key, value)
size := msgSize(key, value)
writeInt64(w, offset)
writeInt32(w, size)
writeInt32(w, int32(crc32))
writeInt8(w, magicByte)
writeInt8(w, attributes)
writeInt64(w, timestamp)
writeBytes(w, key)
writeBytes(w, value)
}
func msgSize(key, value []byte) int32 {
return 4 + // crc
1 + // magic byte
1 + // attributes
8 + // timestamp
sizeofBytes(key) +
sizeofBytes(value)
}
// Messages with magic >2 are called records. This method writes messages using message format 2.
func writeRecord(w *bufio.Writer, attributes int8, baseTime time.Time, baseOffset int64, msg Message) {
timestampDelta := msg.Time.Sub(baseTime)
offsetDelta := int64(msg.Offset - baseOffset)
writeVarInt(w, int64(recordSize(&msg, timestampDelta, offsetDelta)))
writeInt8(w, attributes)
writeVarInt(w, int64(timestampDelta))
writeVarInt(w, offsetDelta)
writeVarInt(w, int64(len(msg.Key)))
w.Write(msg.Key)
writeVarInt(w, int64(len(msg.Value)))
w.Write(msg.Value)
writeVarInt(w, int64(len(msg.Headers)))
for _, h := range msg.Headers {
writeVarInt(w, int64(len(h.Key)))
w.Write([]byte(h.Key))
writeVarInt(w, int64(len(h.Value)))
w.Write(h.Value)
}
}