forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
record_v1.go
243 lines (203 loc) · 5.21 KB
/
record_v1.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
package protocol
import (
"errors"
"hash/crc32"
"io"
"math"
"time"
)
func readMessage(b *pageBuffer, d *decoder) (attributes int8, baseOffset, timestamp int64, key, value Bytes, err error) {
md := decoder{
reader: d,
remain: 12,
}
baseOffset = md.readInt64()
md.remain = int(md.readInt32())
crc := uint32(md.readInt32())
md.setCRC(crc32.IEEETable)
magicByte := md.readInt8()
attributes = md.readInt8()
timestamp = int64(0)
if magicByte != 0 {
timestamp = md.readInt64()
}
keyOffset := b.Size()
keyLength := int(md.readInt32())
hasKey := keyLength >= 0
if hasKey {
md.writeTo(b, keyLength)
key = b.ref(keyOffset, b.Size())
}
valueOffset := b.Size()
valueLength := int(md.readInt32())
hasValue := valueLength >= 0
if hasValue {
md.writeTo(b, valueLength)
value = b.ref(valueOffset, b.Size())
}
if md.crc32 != crc {
err = Errorf("crc32 checksum mismatch (computed=%d found=%d)", md.crc32, crc)
} else {
err = dontExpectEOF(md.err)
}
return
}
func (rs *RecordSet) readFromVersion1(d *decoder) error {
var records RecordReader
b := newPageBuffer()
defer b.unref()
attributes, baseOffset, timestamp, key, value, err := readMessage(b, d)
if err != nil {
return err
}
if compression := Attributes(attributes).Compression(); compression == 0 {
records = &message{
Record: Record{
Offset: baseOffset,
Time: makeTime(timestamp),
Key: key,
Value: value,
},
}
} else {
// Can we have a non-nil key when reading a compressed message?
if key != nil {
key.Close()
}
if value == nil {
records = emptyRecordReader{}
} else {
defer value.Close()
codec := compression.Codec()
if codec == nil {
return Errorf("unsupported compression codec: %d", compression)
}
decompressor := codec.NewReader(value)
defer decompressor.Close()
b := newPageBuffer()
defer b.unref()
d := &decoder{
reader: decompressor,
remain: math.MaxInt32,
}
r := &recordReader{
records: make([]Record, 0, 32),
}
for !d.done() {
_, offset, timestamp, key, value, err := readMessage(b, d)
if err != nil {
if errors.Is(err, io.ErrUnexpectedEOF) {
break
}
for _, rec := range r.records {
closeBytes(rec.Key)
closeBytes(rec.Value)
}
return err
}
r.records = append(r.records, Record{
Offset: offset,
Time: makeTime(timestamp),
Key: key,
Value: value,
})
}
if baseOffset != 0 {
// https://kafka.apache.org/documentation/#messageset
//
// In version 1, to avoid server side re-compression, only the
// wrapper message will be assigned an offset. The inner messages
// will have relative offsets. The absolute offset can be computed
// using the offset from the outer message, which corresponds to the
// offset assigned to the last inner message.
lastRelativeOffset := int64(len(r.records)) - 1
for i := range r.records {
r.records[i].Offset = baseOffset - (lastRelativeOffset - r.records[i].Offset)
}
}
records = r
}
}
*rs = RecordSet{
Version: 1,
Attributes: Attributes(attributes),
Records: records,
}
return nil
}
func (rs *RecordSet) writeToVersion1(buffer *pageBuffer, bufferOffset int64) error {
attributes := rs.Attributes
records := rs.Records
if compression := attributes.Compression(); compression != 0 {
if codec := compression.Codec(); codec != nil {
// In the message format version 1, compression is achieved by
// compressing the value of a message which recursively contains
// the representation of the compressed message set.
subset := *rs
subset.Attributes &= ^7 // erase compression
if err := subset.writeToVersion1(buffer, bufferOffset); err != nil {
return err
}
compressed := newPageBuffer()
defer compressed.unref()
compressor := codec.NewWriter(compressed)
defer compressor.Close()
var err error
buffer.pages.scan(bufferOffset, buffer.Size(), func(b []byte) bool {
_, err = compressor.Write(b)
return err == nil
})
if err != nil {
return err
}
if err := compressor.Close(); err != nil {
return err
}
buffer.Truncate(int(bufferOffset))
records = &message{
Record: Record{
Value: compressed,
},
}
}
}
e := encoder{writer: buffer}
currentTimestamp := timestamp(time.Now())
return forEachRecord(records, func(i int, r *Record) error {
t := timestamp(r.Time)
if t == 0 {
t = currentTimestamp
}
messageOffset := buffer.Size()
e.writeInt64(int64(i))
e.writeInt32(0) // message size placeholder
e.writeInt32(0) // crc32 placeholder
e.setCRC(crc32.IEEETable)
e.writeInt8(1) // magic byte: version 1
e.writeInt8(int8(attributes))
e.writeInt64(t)
if err := e.writeNullBytesFrom(r.Key); err != nil {
return err
}
if err := e.writeNullBytesFrom(r.Value); err != nil {
return err
}
b0 := packUint32(uint32(buffer.Size() - (messageOffset + 12)))
b1 := packUint32(e.crc32)
buffer.WriteAt(b0[:], messageOffset+8)
buffer.WriteAt(b1[:], messageOffset+12)
e.setCRC(nil)
return nil
})
}
type message struct {
Record Record
read bool
}
func (m *message) ReadRecord() (*Record, error) {
if m.read {
return nil, io.EOF
}
m.read = true
return &m.Record, nil
}