@@ -84,7 +84,7 @@ func (f v0MessageSetBuilder) messages() []Message {
84
84
}
85
85
86
86
func (f v0MessageSetBuilder ) bytes () []byte {
87
- return newWB ().call (func (wb * kafkaWriteBuffer ) {
87
+ bs := newWB ().call (func (wb * kafkaWriteBuffer ) {
88
88
for _ , msg := range f .msgs {
89
89
bs := newWB ().call (func (wb * kafkaWriteBuffer ) {
90
90
wb .writeInt64 (msg .Offset ) // offset
@@ -96,22 +96,23 @@ func (f v0MessageSetBuilder) bytes() []byte {
96
96
wb .writeBytes (msg .Value )
97
97
}))
98
98
})
99
- if f .codec != nil {
100
- bs = newWB ().call (func (wb * kafkaWriteBuffer ) {
101
- wb .writeInt64 (msg .Offset ) // offset
102
- wb .writeBytes (newWB ().call (func (wb * kafkaWriteBuffer ) {
103
- compressed := mustCompress (bs , f .codec )
104
- wb .writeInt32 (- 1 ) // crc, unused
105
- wb .writeInt8 (0 ) // magic
106
- wb .writeInt8 (f .codec .Code ()) // attributes
107
- wb .writeBytes (nil ) // key is always nil for compressed
108
- wb .writeBytes (compressed ) // the value is the compressed message
109
- }))
110
- })
111
- }
112
99
wb .Write (bs )
113
100
}
114
101
})
102
+ if f .codec != nil {
103
+ bs = newWB ().call (func (wb * kafkaWriteBuffer ) {
104
+ wb .writeInt64 (f .msgs [0 ].Offset ) // offset
105
+ wb .writeBytes (newWB ().call (func (wb * kafkaWriteBuffer ) {
106
+ compressed := mustCompress (bs , f .codec )
107
+ wb .writeInt32 (- 1 ) // crc, unused
108
+ wb .writeInt8 (0 ) // magic
109
+ wb .writeInt8 (f .codec .Code ()) // attributes
110
+ wb .writeBytes (nil ) // key is always nil for compressed
111
+ wb .writeBytes (compressed ) // the value is the compressed message
112
+ }))
113
+ })
114
+ }
115
+ return bs
115
116
}
116
117
117
118
type v1MessageSetBuilder struct {
@@ -124,10 +125,14 @@ func (f v1MessageSetBuilder) messages() []Message {
124
125
}
125
126
126
127
func (f v1MessageSetBuilder ) bytes () []byte {
127
- return newWB ().call (func (wb * kafkaWriteBuffer ) {
128
- for _ , msg := range f .msgs {
128
+ bs := newWB ().call (func (wb * kafkaWriteBuffer ) {
129
+ for i , msg := range f .msgs {
129
130
bs := newWB ().call (func (wb * kafkaWriteBuffer ) {
130
- wb .writeInt64 (msg .Offset ) // offset
131
+ if f .codec != nil {
132
+ wb .writeInt64 (int64 (i )) // compressed inner message offsets are relative
133
+ } else {
134
+ wb .writeInt64 (msg .Offset ) // offset
135
+ }
131
136
wb .writeBytes (newWB ().call (func (wb * kafkaWriteBuffer ) {
132
137
wb .writeInt32 (- 1 ) // crc, unused
133
138
wb .writeInt8 (1 ) // magic
@@ -137,23 +142,24 @@ func (f v1MessageSetBuilder) bytes() []byte {
137
142
wb .writeBytes (msg .Value )
138
143
}))
139
144
})
140
- if f .codec != nil {
141
- bs = newWB ().call (func (wb * kafkaWriteBuffer ) {
142
- wb .writeInt64 (msg .Offset ) // offset
143
- wb .writeBytes (newWB ().call (func (wb * kafkaWriteBuffer ) {
144
- bs := mustCompress (bs , f .codec )
145
- wb .writeInt32 (- 1 ) // crc, unused
146
- wb .writeInt8 (1 ) // magic
147
- wb .writeInt8 (f .codec .Code ()) // attributes
148
- wb .writeInt64 (msg .Time .UnixMilli ()) // timestamp
149
- wb .writeBytes (nil ) // key is always nil for compressed
150
- wb .writeBytes (bs ) // the value is the compressed message
151
- }))
152
- })
153
- }
154
145
wb .Write (bs )
155
146
}
156
147
})
148
+ if f .codec != nil {
149
+ bs = newWB ().call (func (wb * kafkaWriteBuffer ) {
150
+ wb .writeInt64 (f .msgs [len (f .msgs )- 1 ].Offset ) // offset of the wrapper message is the last offset of the inner messages
151
+ wb .writeBytes (newWB ().call (func (wb * kafkaWriteBuffer ) {
152
+ bs := mustCompress (bs , f .codec )
153
+ wb .writeInt32 (- 1 ) // crc, unused
154
+ wb .writeInt8 (1 ) // magic
155
+ wb .writeInt8 (f .codec .Code ()) // attributes
156
+ wb .writeInt64 (f .msgs [0 ].Time .UnixMilli ()) // timestamp
157
+ wb .writeBytes (nil ) // key is always nil for compressed
158
+ wb .writeBytes (bs ) // the value is the compressed message
159
+ }))
160
+ })
161
+ }
162
+ return bs
157
163
}
158
164
159
165
type v2MessageSetBuilder struct {
0 commit comments