forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
listoffset.go
286 lines (235 loc) · 7.15 KB
/
listoffset.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
package skafka
import (
"bufio"
"context"
"fmt"
"net"
"time"
"github.com/segmentio/kafka-go/protocol/listoffsets"
)
// OffsetRequest represents a request to retrieve a single partition offset.
type OffsetRequest struct {
Partition int
Timestamp int64
}
// FirstOffsetOf constructs an OffsetRequest which asks for the first offset of
// the parition given as argument.
func FirstOffsetOf(partition int) OffsetRequest {
return OffsetRequest{Partition: partition, Timestamp: FirstOffset}
}
// LastOffsetOf constructs an OffsetRequest which asks for the last offset of
// the partition given as argument.
func LastOffsetOf(partition int) OffsetRequest {
return OffsetRequest{Partition: partition, Timestamp: LastOffset}
}
// TimeOffsetOf constructs an OffsetRequest which asks for a partition offset
// at a given time.
func TimeOffsetOf(partition int, at time.Time) OffsetRequest {
return OffsetRequest{Partition: partition, Timestamp: timestamp(at)}
}
// PartitionOffsets carries information about offsets available in a topic
// partition.
type PartitionOffsets struct {
Partition int
FirstOffset int64
LastOffset int64
Offsets map[int64]time.Time
Error error
}
// ListOffsetsRequest represents a request sent to a kafka broker to list of the
// offsets of topic partitions.
type ListOffsetsRequest struct {
// Address of the kafka broker to send the request to.
Addr net.Addr
// A mapping of topic names to list of partitions that the program wishes to
// get the offsets for.
Topics map[string][]OffsetRequest
// The isolation level for the request.
//
// Defaults to ReadUncommitted.
//
// This field requires the kafka broker to support the ListOffsets API in
// version 2 or above (otherwise the value is ignored).
IsolationLevel IsolationLevel
}
// ListOffsetsResponse represents a response from a kafka broker to a offset
// listing request.
type ListOffsetsResponse struct {
// The amount of time that the broker throttled the request.
Throttle time.Duration
// Mappings of topics names to partition offsets, there will be one entry
// for each topic in the request.
Topics map[string][]PartitionOffsets
}
// ListOffsets sends an offset request to a kafka broker and returns the
// response.
func (c *Client) ListOffsets(ctx context.Context, req *ListOffsetsRequest) (*ListOffsetsResponse, error) {
type topicPartition struct {
topic string
partition int
}
partitionOffsets := make(map[topicPartition]PartitionOffsets)
for topicName, requests := range req.Topics {
for _, r := range requests {
key := topicPartition{
topic: topicName,
partition: r.Partition,
}
partition, ok := partitionOffsets[key]
if !ok {
partition = PartitionOffsets{
Partition: r.Partition,
FirstOffset: -1,
LastOffset: -1,
Offsets: make(map[int64]time.Time),
}
}
switch r.Timestamp {
case FirstOffset:
partition.FirstOffset = 0
case LastOffset:
partition.LastOffset = 0
}
partitionOffsets[topicPartition{
topic: topicName,
partition: r.Partition,
}] = partition
}
}
topics := make([]listoffsets.RequestTopic, 0, len(req.Topics))
for topicName, requests := range req.Topics {
partitions := make([]listoffsets.RequestPartition, len(requests))
for i, r := range requests {
partitions[i] = listoffsets.RequestPartition{
Partition: int32(r.Partition),
CurrentLeaderEpoch: -1,
Timestamp: r.Timestamp,
}
}
topics = append(topics, listoffsets.RequestTopic{
Topic: topicName,
Partitions: partitions,
})
}
m, err := c.roundTrip(ctx, req.Addr, &listoffsets.Request{
ReplicaID: -1,
IsolationLevel: int8(req.IsolationLevel),
Topics: topics,
})
if err != nil {
return nil, fmt.Errorf("kafka.(*Client).ListOffsets: %w", err)
}
res := m.(*listoffsets.Response)
ret := &ListOffsetsResponse{
Throttle: makeDuration(res.ThrottleTimeMs),
Topics: make(map[string][]PartitionOffsets, len(res.Topics)),
}
for _, t := range res.Topics {
for _, p := range t.Partitions {
key := topicPartition{
topic: t.Topic,
partition: int(p.Partition),
}
partition := partitionOffsets[key]
switch p.Timestamp {
case FirstOffset:
partition.FirstOffset = p.Offset
case LastOffset:
partition.LastOffset = p.Offset
default:
partition.Offsets[p.Offset] = makeTime(p.Timestamp)
}
if p.ErrorCode != 0 {
partition.Error = Error(p.ErrorCode)
}
partitionOffsets[key] = partition
}
}
for key, partition := range partitionOffsets {
ret.Topics[key.topic] = append(ret.Topics[key.topic], partition)
}
return ret, nil
}
type listOffsetRequestV1 struct {
ReplicaID int32
Topics []listOffsetRequestTopicV1
}
func (r listOffsetRequestV1) size() int32 {
return 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() })
}
func (r listOffsetRequestV1) writeTo(wb *writeBuffer) {
wb.writeInt32(r.ReplicaID)
wb.writeArray(len(r.Topics), func(i int) { r.Topics[i].writeTo(wb) })
}
type listOffsetRequestTopicV1 struct {
TopicName string
Partitions []listOffsetRequestPartitionV1
}
func (t listOffsetRequestTopicV1) size() int32 {
return sizeofString(t.TopicName) +
sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() })
}
func (t listOffsetRequestTopicV1) writeTo(wb *writeBuffer) {
wb.writeString(t.TopicName)
wb.writeArray(len(t.Partitions), func(i int) { t.Partitions[i].writeTo(wb) })
}
type listOffsetRequestPartitionV1 struct {
Partition int32
Time int64
}
func (p listOffsetRequestPartitionV1) size() int32 {
return 4 + 8
}
func (p listOffsetRequestPartitionV1) writeTo(wb *writeBuffer) {
wb.writeInt32(p.Partition)
wb.writeInt64(p.Time)
}
type listOffsetResponseV1 []listOffsetResponseTopicV1
func (r listOffsetResponseV1) size() int32 {
return sizeofArray(len(r), func(i int) int32 { return r[i].size() })
}
func (r listOffsetResponseV1) writeTo(wb *writeBuffer) {
wb.writeArray(len(r), func(i int) { r[i].writeTo(wb) })
}
type listOffsetResponseTopicV1 struct {
TopicName string
PartitionOffsets []partitionOffsetV1
}
func (t listOffsetResponseTopicV1) size() int32 {
return sizeofString(t.TopicName) +
sizeofArray(len(t.PartitionOffsets), func(i int) int32 { return t.PartitionOffsets[i].size() })
}
func (t listOffsetResponseTopicV1) writeTo(wb *writeBuffer) {
wb.writeString(t.TopicName)
wb.writeArray(len(t.PartitionOffsets), func(i int) { t.PartitionOffsets[i].writeTo(wb) })
}
type partitionOffsetV1 struct {
Partition int32
ErrorCode int16
Timestamp int64
Offset int64
}
func (p partitionOffsetV1) size() int32 {
return 4 + 2 + 8 + 8
}
func (p partitionOffsetV1) writeTo(wb *writeBuffer) {
wb.writeInt32(p.Partition)
wb.writeInt16(p.ErrorCode)
wb.writeInt64(p.Timestamp)
wb.writeInt64(p.Offset)
}
func (p *partitionOffsetV1) readFrom(r *bufio.Reader, sz int) (remain int, err error) {
if remain, err = readInt32(r, sz, &p.Partition); err != nil {
return
}
if remain, err = readInt16(r, remain, &p.ErrorCode); err != nil {
return
}
if remain, err = readInt64(r, remain, &p.Timestamp); err != nil {
return
}
if remain, err = readInt64(r, remain, &p.Offset); err != nil {
return
}
return
}