forked from pingcap/tidb-tools
-
Notifications
You must be signed in to change notification settings - Fork 0
/
offset.go
172 lines (146 loc) · 4.06 KB
/
offset.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
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package reader
import (
"github.com/Shopify/sarama"
"github.com/pingcap/errors"
pb "github.com/pingcap/tidb-tools/tidb-binlog/slave_binlog_proto/go-binlog"
log "github.com/sirupsen/logrus"
)
// KafkaSeeker seeks offset in kafka topics by given condition
type KafkaSeeker struct {
consumer sarama.Consumer
client sarama.Client
}
// NewKafkaSeeker creates an instance of KafkaSeeker
func NewKafkaSeeker(addr []string, config *sarama.Config) (*KafkaSeeker, error) {
client, err := sarama.NewClient(addr, config)
if err != nil {
return nil, errors.Trace(err)
}
consumer, err := sarama.NewConsumerFromClient(client)
if err != nil {
return nil, errors.Trace(err)
}
s := &KafkaSeeker{
client: client,
consumer: consumer,
}
return s, nil
}
// Close releases resources of KafkaSeeker
func (ks *KafkaSeeker) Close() {
ks.consumer.Close()
ks.client.Close()
}
// Seek seeks the first offset which binlog CommitTs bigger than ts
func (ks *KafkaSeeker) Seek(topic string, ts int64, partitions []int32) (offsets []int64, err error) {
if len(partitions) == 0 {
partitions, err = ks.consumer.Partitions(topic)
if err != nil {
log.Errorf("get partitions from topic %s error %v", topic, err)
return nil, errors.Trace(err)
}
}
offsets, err = ks.seekOffsets(topic, partitions, ts)
if err != nil {
err = errors.Trace(err)
log.Errorf("seek offsets error %v", err)
}
return
}
func (ks *KafkaSeeker) getTSFromMSG(msg *sarama.ConsumerMessage) (ts int64, err error) {
binlog := new(pb.Binlog)
err = binlog.Unmarshal(msg.Value)
if err != nil {
err = errors.Trace(err)
return
}
return binlog.CommitTs, nil
}
// seekOffsets returns all valid offsets in partitions
func (ks *KafkaSeeker) seekOffsets(topic string, partitions []int32, pos int64) ([]int64, error) {
offsets := make([]int64, len(partitions))
for _, partition := range partitions {
start, err := ks.client.GetOffset(topic, partition, sarama.OffsetOldest)
if err != nil {
err = errors.Trace(err)
return nil, err
}
end, err := ks.client.GetOffset(topic, partition, sarama.OffsetNewest)
if err != nil {
err = errors.Trace(err)
return nil, err
}
offset, err := ks.seekOffset(topic, partition, start, end-1, pos)
if err != nil {
err = errors.Trace(err)
return nil, err
}
offsets[partition] = offset
}
return offsets, nil
}
func (ks *KafkaSeeker) seekOffset(topic string, partition int32, start int64, end int64, ts int64) (offset int64, err error) {
startTS, err := ks.getTSAtOffset(topic, partition, start)
if err != nil {
err = errors.Trace(err)
return
}
if ts < startTS {
log.Warnf("given ts %v is smaller than oldest message's ts %v, some binlogs may lose", ts, startTS)
offset = start
return
} else if ts == startTS {
offset = start + 1
return
}
for start < end {
mid := (end-start)/2 + start
var midTS int64
midTS, err = ks.getTSAtOffset(topic, partition, mid)
if err != nil {
err = errors.Trace(err)
return
}
if midTS <= ts {
start = mid + 1
} else {
end = mid
}
}
var endTS int64
endTS, err = ks.getTSAtOffset(topic, partition, end)
if err != nil {
err = errors.Trace(err)
return
}
if endTS <= ts {
return end + 1, nil
}
return end, nil
}
func (ks *KafkaSeeker) getTSAtOffset(topic string, partition int32, offset int64) (ts int64, err error) {
pc, err := ks.consumer.ConsumePartition(topic, partition, offset)
if err != nil {
err = errors.Trace(err)
return
}
defer pc.Close()
for msg := range pc.Messages() {
ts, err = ks.getTSFromMSG(msg)
err = errors.Trace(err)
return
}
panic("unreachable")
}