forked from Shopify/ghostferry
-
Notifications
You must be signed in to change notification settings - Fork 1
/
Copy pathbinlog_streamer.go
445 lines (377 loc) · 13.7 KB
/
binlog_streamer.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
package ghostferry
import (
"context"
"crypto/tls"
sqlorig "database/sql"
"fmt"
sql "github.com/Shopify/ghostferry/sqlwrapper"
"time"
"github.com/siddontang/go-mysql/mysql"
"github.com/siddontang/go-mysql/replication"
"github.com/sirupsen/logrus"
)
const caughtUpThreshold = 10 * time.Second
type BinlogPosition struct {
// A binlog position emitted by the binlog-streamer consists of two parts:
// First, the last emitted event position, which refers to the event that
// we received from the MySQL master and that we hand to clients. Second,
// a position from which we can resume a binlog-streamer.
// Ideally, these two values would be the same, but in reality they are
// not, because some events are streamed in a series (e.g. DML events
// require a table-map events to be seen before).
// As a result, we always stream event positions as a pair - if a binlog
// streamer is resumed from an event that is not safe to resume from, we
// resume from the most recent (earlier) event from which we can safely
// resume and simply suppress emitting these events up to the point of the
// last event returned.
//
// the actual binlog position of an event emitted by the streamer
EventPosition mysql.Position
// the position from which one needs to point the streamer if we want to
// resume from after this event
ResumePosition mysql.Position
}
func NewResumableBinlogPosition(pos mysql.Position) BinlogPosition {
return BinlogPosition{pos, pos}
}
func (p BinlogPosition) Compare(o BinlogPosition) int {
// comparison always happens on the actual event
return p.EventPosition.Compare(o.EventPosition)
}
func (b BinlogPosition) String() string {
return fmt.Sprintf("Position(event %s, resume at %s)", b.EventPosition, b.ResumePosition)
}
type ReplicationEvent struct {
BinlogPosition BinlogPosition
BinlogEvent *replication.BinlogEvent
EventTime time.Time
}
type BinlogStreamer struct {
DB *sql.DB
DBConfig *DatabaseConfig
MyServerId uint32
ErrorHandler ErrorHandler
ReadRetries int
binlogSyncer *replication.BinlogSyncer
binlogStreamer *replication.BinlogStreamer
// what is the last event that we ever received from the streamer
lastStreamedBinlogPosition mysql.Position
// what is the last event that we received and from which it is possible
// to resume
lastResumeBinlogPosition mysql.Position
// if we have resumed from an earlier position than where we last streamed
// to (that is, if lastResumeBinlogPosition is before
// lastStreamedBinlogPosition when resuming), up to what event should we
// suppress emitting events
suppressEmitUpToBinlogPosition mysql.Position
// up to what position to we want to continue streaming (if a stop was
// requested)
targetBinlogPosition mysql.Position
lastProcessedEventTime time.Time
lastLagMetricEmittedTime time.Time
stopRequested bool
logger *logrus.Entry
eventListeners []func(*ReplicationEvent) error
}
func (s *BinlogStreamer) ensureLogger() {
if s.logger == nil {
s.logger = logrus.WithField("tag", "binlog_streamer")
}
}
func (s *BinlogStreamer) createBinlogSyncer() error {
var err error
var tlsConfig *tls.Config
if s.DBConfig.TLS != nil {
tlsConfig, err = s.DBConfig.TLS.BuildConfig()
if err != nil {
return err
}
}
if s.MyServerId == 0 {
s.MyServerId, err = s.generateNewServerId()
if err != nil {
s.logger.WithError(err).Error("could not generate unique server_id")
return err
}
}
syncerConfig := replication.BinlogSyncerConfig{
ServerID: s.MyServerId,
Host: s.DBConfig.Host,
Port: s.DBConfig.Port,
User: s.DBConfig.User,
Password: s.DBConfig.Pass,
TLSConfig: tlsConfig,
UseDecimal: true,
TimestampStringLocation: time.UTC,
}
s.binlogSyncer = replication.NewBinlogSyncer(syncerConfig)
return nil
}
func (s *BinlogStreamer) ConnectBinlogStreamerToMysql() (BinlogPosition, error) {
s.ensureLogger()
currentPosition, err := ShowMasterStatusBinlogPosition(s.DB)
if err != nil {
s.logger.WithError(err).Error("failed to read current binlog position")
return BinlogPosition{}, err
}
s.logger.Debugf("connecting to binlog streamer using master state %s", currentPosition)
return s.ConnectBinlogStreamerToMysqlFrom(NewResumableBinlogPosition(currentPosition))
}
func (s *BinlogStreamer) ConnectBinlogStreamerToMysqlFrom(startFromBinlogPosition BinlogPosition) (BinlogPosition, error) {
s.ensureLogger()
err := s.createBinlogSyncer()
if err != nil {
return BinlogPosition{}, err
}
if startFromBinlogPosition.EventPosition.Compare(startFromBinlogPosition.ResumePosition) < 0 {
err = fmt.Errorf("invalid resume position %s: last event must not be before resume position", startFromBinlogPosition)
return BinlogPosition{}, err
}
s.lastStreamedBinlogPosition = startFromBinlogPosition.EventPosition
s.suppressEmitUpToBinlogPosition = startFromBinlogPosition.EventPosition
s.lastResumeBinlogPosition = startFromBinlogPosition.ResumePosition
s.logger.WithFields(logrus.Fields{
"stream.file": s.lastStreamedBinlogPosition.Name,
"stream.pos": s.lastStreamedBinlogPosition.Pos,
"resume.file": s.lastResumeBinlogPosition.Name,
"resume.pos": s.lastResumeBinlogPosition.Pos,
}).Info("starting binlog streaming")
s.binlogStreamer, err = s.binlogSyncer.StartSync(s.lastResumeBinlogPosition)
if err != nil {
s.logger.WithError(err).Error("unable to start binlog streamer")
return BinlogPosition{}, err
}
return startFromBinlogPosition, err
}
func (s *BinlogStreamer) Run() {
s.ensureLogger()
defer func() {
s.logger.Info("exiting binlog streamer")
s.binlogSyncer.Close()
}()
s.logger.Info("starting binlog streamer")
for !s.stopRequested || (s.stopRequested && s.lastStreamedBinlogPosition.Compare(s.targetBinlogPosition) < 0) {
var ev *replication.BinlogEvent
var timedOut bool
err := WithRetries(s.ReadRetries, 0, s.logger, "get binlog event", func() (er error) {
ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond)
defer cancel()
ev, er = s.binlogStreamer.GetEvent(ctx)
if er == context.DeadlineExceeded {
timedOut = true
return nil
}
return er
})
if err != nil {
s.ErrorHandler.Fatal("binlog_streamer", err)
}
if timedOut {
s.lastProcessedEventTime = time.Now()
continue
}
if IncrediblyVerboseLogging {
s.logger.WithFields(logrus.Fields{
"header": ev.Header,
"event": ev.Event,
"raw": ev.RawData,
}).Debugf("Received %T event from binlog", ev);
if tableMapEvent, ok := ev.Event.(*replication.TableMapEvent); ok {
s.logger.Debugf("Event is table-map event for table %d", tableMapEvent.TableID);
}
}
switch e := ev.Event.(type) {
case *replication.RotateEvent:
// This event is needed because we need to update the last successful
// binlog position.
s.lastStreamedBinlogPosition.Pos = uint32(e.Position)
s.lastStreamedBinlogPosition.Name = string(e.NextLogName)
s.logger.WithFields(logrus.Fields{
"pos": s.lastStreamedBinlogPosition.Pos,
"file": s.lastStreamedBinlogPosition.Name,
}).Info("rotated binlog file")
case *replication.RowsEvent:
err = s.emitEvent(ev)
if err != nil {
s.logger.WithError(err).Error("failed to handle rows event")
s.ErrorHandler.Fatal("binlog_streamer", err)
}
s.updateLastStreamedPosAndTime(ev)
case *replication.FormatDescriptionEvent:
// This event has a LogPos = 0, presumably because this is the first
// event received by the BinlogStreamer to get some metadata about
// how the binlog is supposed to be transmitted.
// We don't want to save the binlog position derived from this event
// as it will contain the wrong thing.
continue
case *replication.QueryEvent:
// This event tells us about table structure change which means
// the cached schemas of the tables would be invalidated.
err = s.emitEvent(ev)
if err != nil {
s.logger.WithError(err).Error("failed to handle query event")
s.ErrorHandler.Fatal("binlog_streamer", err)
}
s.updateLastStreamedPosAndTime(ev)
case *replication.GenericEvent:
// go-mysql don't parse all events and unparsed events are denoted
// with empty GenericEvent structs.
// so there's no way to handle this for us.
continue
default:
s.updateLastStreamedPosAndTime(ev)
}
}
s.logger.Info("binlog streamer stopped")
}
func (s *BinlogStreamer) AddEventListener(listener func(*ReplicationEvent) error) {
s.eventListeners = append(s.eventListeners, listener)
}
func (s *BinlogStreamer) GetLastStreamedBinlogPosition() mysql.Position {
return s.lastStreamedBinlogPosition
}
func (s *BinlogStreamer) IsAlmostCaughtUp() bool {
return time.Now().Sub(s.lastProcessedEventTime) < caughtUpThreshold
}
func (s *BinlogStreamer) FlushAndStop() {
s.logger.Info("requesting binlog streamer to stop")
// Must first read the binlog position before requesting stop
// Otherwise there is a race condition where the stopRequested is
// set to True but the TargetPosition is nil, which would cause
// the BinlogStreamer to immediately exit, as it thinks that it has
// passed the initial target position.
err := WithRetries(100, 600*time.Millisecond, s.logger, "read current binlog position", func() error {
var err error
s.targetBinlogPosition, err = ShowMasterStatusBinlogPosition(s.DB)
return err
})
if err != nil {
s.ErrorHandler.Fatal("binlog_streamer", err)
}
s.logger.WithField("target_position", s.targetBinlogPosition).Info("current stop binlog position was recorded")
s.stopRequested = true
}
func (s *BinlogStreamer) updateLastStreamedPosAndTime(ev *replication.BinlogEvent) {
if ev.Header.LogPos == 0 || ev.Header.Timestamp == 0 {
// This shouldn't happen, as the cases where it does happen are excluded.
// However, I've not seen all the cases yet.
s.logger.Panicf("logpos: %d %d %T", ev.Header.LogPos, ev.Header.Timestamp, ev.Event)
}
s.lastStreamedBinlogPosition.Pos = ev.Header.LogPos
eventTime := time.Unix(int64(ev.Header.Timestamp), 0)
s.lastProcessedEventTime = eventTime
if resumablePosition, evIsResumable := s.getResumePositionForEvent(ev); evIsResumable {
s.lastResumeBinlogPosition = resumablePosition
}
if time.Since(s.lastLagMetricEmittedTime) >= time.Second {
lag := time.Since(eventTime)
metrics.Gauge("BinlogStreamer.Lag", lag.Seconds(), nil, 1.0)
s.lastLagMetricEmittedTime = time.Now()
}
}
func (s *BinlogStreamer) getResumePositionForEvent(ev *replication.BinlogEvent) (resumablePosition mysql.Position, evIsResumable bool) {
// resuming from a RowsEvent is not possible, as it may be followed by
// another rows-event without a subsequent TableMapEvent. Thus, if we have
// a rows-event, we need to keep resuming from whatever last non-rows-event
//
// The same is true for TableMapEvents themselves, as we cannot resume right
// after the event: we need to re-stream the event itself to get ready for
// a RowsEvent
switch ev.Event.(type) {
case *replication.RowsEvent, *replication.TableMapEvent:
// it's not resumable - we need to return whatever was save to resume
// from before
resumablePosition = s.lastResumeBinlogPosition
default:
// it is safe to resume from here
evIsResumable = true
resumablePosition = mysql.Position{
// The filename is only changed and visible during the RotateEvent, which
// is handled transparently in Run().
Name: s.lastStreamedBinlogPosition.Name,
Pos: ev.Header.LogPos,
}
}
return
}
func (s *BinlogStreamer) emitEvent(ev *replication.BinlogEvent) error {
if ev.Header.LogPos == 0 {
// This shouldn't happen, as rows events always have a logpos.
return fmt.Errorf("invalid replication event logpos: %d %d %T", ev.Header.LogPos, ev.Header.Timestamp, ev.Event)
}
pos := mysql.Position{
// The filename is only changed and visible during the RotateEvent, which
// is handled transparently in Run().
Name: s.lastStreamedBinlogPosition.Name,
Pos: ev.Header.LogPos,
}
// we may still be searching for the first event to stream to listeners, if
// we resumed reading upstream events from an earlier event
if pos.Compare(s.suppressEmitUpToBinlogPosition) <= 0 {
if IncrediblyVerboseLogging {
s.logger.Debugf("Skip emitting event at binlog position %v: waiting for event %v", pos, s.suppressEmitUpToBinlogPosition);
}
return nil
}
resumePosition, _ := s.getResumePositionForEvent(ev)
event := &ReplicationEvent{
BinlogPosition: BinlogPosition{
EventPosition: pos,
ResumePosition: resumePosition,
},
BinlogEvent: ev,
EventTime: time.Unix(int64(ev.Header.Timestamp), 0),
}
for _, listener := range s.eventListeners {
err := listener(event)
if err != nil {
return err
}
}
return nil
}
func (s *BinlogStreamer) generateNewServerId() (uint32, error) {
var id uint32
for {
id = randomServerId()
exists, err := idExistsOnServer(id, s.DB)
if err != nil {
return 0, err
}
if !exists {
break
}
s.logger.WithField("server_id", id).Warn("server_id was taken, retrying")
}
return id, nil
}
func idExistsOnServer(id uint32, db *sql.DB) (bool, error) {
curIds, err := idsOnServer(db)
if err != nil {
return false, err
}
for _, idd := range curIds {
if idd == id {
return true, nil
}
}
return false, nil
}
func idsOnServer(db *sql.DB) ([]uint32, error) {
rows, err := db.Query("SHOW SLAVE HOSTS")
if err != nil {
return nil, fmt.Errorf("could not get slave hosts: %s", err)
}
defer rows.Close()
server_ids := make([]uint32, 0)
for rows.Next() {
var server_id uint32
var host, port, master_id, slave_uuid sqlorig.NullString
err = rows.Scan(&server_id, &host, &port, &master_id, &slave_uuid)
if err != nil {
return nil, fmt.Errorf("could not scan SHOW SLAVE HOSTS row, err: %s", err.Error())
}
server_ids = append(server_ids, server_id)
}
return server_ids, nil
}