forked from vitessio/vitess
-
Notifications
You must be signed in to change notification settings - Fork 1
/
binlog_player.go
503 lines (456 loc) · 17.1 KB
/
binlog_player.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
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// Package binlogplayer contains the code that plays a filtered replication
// stream on a client database. It usually runs inside the destination master
// vttablet process.
package binlogplayer
import (
"encoding/hex"
"flag"
"fmt"
"sync"
"time"
"golang.org/x/net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/sqldb"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/stats"
"github.com/youtube/vitess/go/sync2"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
binlogdatapb "github.com/youtube/vitess/go/vt/proto/binlogdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
"github.com/youtube/vitess/go/vt/throttler"
)
var (
// BinlogPlayerConnTimeout is the flag for binlog player connection
// timeout. It is public so the discovery module can also use it.
BinlogPlayerConnTimeout = flag.Duration("binlog_player_conn_timeout", 5*time.Second, "binlog player connection timeout")
// SlowQueryThreshold will cause we logging anything that's higher than it.
SlowQueryThreshold = time.Duration(100 * time.Millisecond)
// keys for the stats map
// BlplQuery is the key for the stats map.
BlplQuery = "Query"
// BlplTransaction is the key for the stats map.
BlplTransaction = "Transaction"
// flags for the blp_checkpoint table. The database entry is just
// a join(",") of these flags.
// BlpFlagDontStart means don't start a BinlogPlayer
BlpFlagDontStart = "DontStart"
)
// Stats is the internal stats of a player. It is a different
// structure that is passed in so stats can be collected over the life
// of multiple individual players.
type Stats struct {
// Stats about the player, keys used are BlplQuery and BlplTransaction
Timings *stats.Timings
Rates *stats.Rates
// Last saved status
lastPosition replication.Position
lastPositionMutex sync.RWMutex
SecondsBehindMaster sync2.AtomicInt64
}
// SetLastPosition sets the last replication position.
func (bps *Stats) SetLastPosition(pos replication.Position) {
bps.lastPositionMutex.Lock()
defer bps.lastPositionMutex.Unlock()
bps.lastPosition = pos
}
// GetLastPosition gets the last replication position.
func (bps *Stats) GetLastPosition() replication.Position {
bps.lastPositionMutex.RLock()
defer bps.lastPositionMutex.RUnlock()
return bps.lastPosition
}
// NewStats creates a new Stats structure
func NewStats() *Stats {
bps := &Stats{}
bps.Timings = stats.NewTimings("")
bps.Rates = stats.NewRates("", bps.Timings, 15, 60e9)
return bps
}
// BinlogPlayer is handling reading a stream of updates from BinlogServer
type BinlogPlayer struct {
tablet *topodatapb.Tablet
dbClient VtClient
// for key range base requests
keyRange *topodatapb.KeyRange
// for table base requests
tables []string
// common to all
uid uint32
position replication.Position
stopPosition replication.Position
blplStats *Stats
defaultCharset *binlogdatapb.Charset
currentCharset *binlogdatapb.Charset
}
// NewBinlogPlayerKeyRange returns a new BinlogPlayer pointing at the server
// replicating the provided keyrange, starting at the startPosition,
// and updating _vt.blp_checkpoint with uid=startPosition.Uid.
// If !stopPosition.IsZero(), it will stop when reaching that position.
func NewBinlogPlayerKeyRange(dbClient VtClient, tablet *topodatapb.Tablet, keyRange *topodatapb.KeyRange, uid uint32, startPosition string, stopPosition string, blplStats *Stats) (*BinlogPlayer, error) {
result := &BinlogPlayer{
tablet: tablet,
dbClient: dbClient,
keyRange: keyRange,
uid: uid,
blplStats: blplStats,
}
var err error
result.position, err = replication.DecodePosition(startPosition)
if err != nil {
return nil, err
}
if stopPosition != "" {
result.stopPosition, err = replication.DecodePosition(stopPosition)
if err != nil {
return nil, err
}
}
return result, nil
}
// NewBinlogPlayerTables returns a new BinlogPlayer pointing at the server
// replicating the provided tables, starting at the startPosition,
// and updating _vt.blp_checkpoint with uid=startPosition.Uid.
// If !stopPosition.IsZero(), it will stop when reaching that position.
func NewBinlogPlayerTables(dbClient VtClient, tablet *topodatapb.Tablet, tables []string, uid uint32, startPosition string, stopPosition string, blplStats *Stats) (*BinlogPlayer, error) {
result := &BinlogPlayer{
tablet: tablet,
dbClient: dbClient,
tables: tables,
uid: uid,
blplStats: blplStats,
}
var err error
result.position, err = replication.DecodePosition(startPosition)
if err != nil {
return nil, err
}
if stopPosition != "" {
var err error
result.stopPosition, err = replication.DecodePosition(stopPosition)
if err != nil {
return nil, err
}
}
return result, nil
}
// writeRecoveryPosition will write the current GTID as the recovery position
// for the next transaction.
// We will also try to get the timestamp for the transaction. Two cases:
// - we have statements, and they start with a SET TIMESTAMP that we
// can parse: then we update transaction_timestamp in blp_checkpoint
// with it, and set SecondsBehindMaster to now() - transaction_timestamp
// - otherwise (the statements are probably filtered out), we leave
// transaction_timestamp alone (keeping the old value), and we don't
// change SecondsBehindMaster
func (blp *BinlogPlayer) writeRecoveryPosition(tx *binlogdatapb.BinlogTransaction) error {
gtid, err := replication.DecodeGTID(tx.TransactionId)
if err != nil {
return err
}
now := time.Now().Unix()
blp.position = replication.AppendGTID(blp.position, gtid)
updateRecovery := updateBlpCheckpoint(blp.uid, blp.position, now, tx.Timestamp)
qr, err := blp.exec(updateRecovery)
if err != nil {
return fmt.Errorf("Error %v in writing recovery info %v", err, updateRecovery)
}
if qr.RowsAffected != 1 {
return fmt.Errorf("Cannot update blp_recovery table, affected %v rows", qr.RowsAffected)
}
blp.blplStats.SetLastPosition(blp.position)
if tx.Timestamp != 0 {
blp.blplStats.SecondsBehindMaster.Set(now - tx.Timestamp)
}
return nil
}
// ReadStartPosition will return the current start position and the flags for
// the provided binlog player.
func ReadStartPosition(dbClient VtClient, uid uint32) (string, string, error) {
selectRecovery := QueryBlpCheckpoint(uid)
qr, err := dbClient.ExecuteFetch(selectRecovery, 1, true)
if err != nil {
return "", "", fmt.Errorf("error %v in selecting from recovery table %v", err, selectRecovery)
}
if qr.RowsAffected != 1 {
return "", "", fmt.Errorf("checkpoint information not available in db for %v", uid)
}
return qr.Rows[0][0].String(), qr.Rows[0][1].String(), nil
}
// readThrottlerSettings will retrieve the throttler settings for filtered
// replication from the checkpoint table.
func (blp *BinlogPlayer) readThrottlerSettings() (int64, int64, error) {
selectThrottlerSettings := QueryBlpThrottlerSettings(blp.uid)
qr, err := blp.dbClient.ExecuteFetch(selectThrottlerSettings, 1, true)
if err != nil {
return throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("error %v in selecting the throttler settings %v", err, selectThrottlerSettings)
}
if qr.RowsAffected != 1 {
return throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("checkpoint information not available in db for %v", blp.uid)
}
maxTPS, err := qr.Rows[0][0].ParseInt64()
if err != nil {
return throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("failed to parse max_tps column: %v", err)
}
maxReplicationLag, err := qr.Rows[0][1].ParseInt64()
if err != nil {
return throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("failed to parse max_replication_lag column: %v", err)
}
return maxTPS, maxReplicationLag, nil
}
func (blp *BinlogPlayer) processTransaction(tx *binlogdatapb.BinlogTransaction) (ok bool, err error) {
txnStartTime := time.Now()
if err = blp.dbClient.Begin(); err != nil {
return false, fmt.Errorf("failed query BEGIN, err: %s", err)
}
if err = blp.writeRecoveryPosition(tx); err != nil {
return false, err
}
for i, stmt := range tx.Statements {
// Make sure the statement is replayed in the proper charset.
if dbClient, ok := blp.dbClient.(*DBClient); ok {
var stmtCharset *binlogdatapb.Charset
if stmt.Charset != nil {
stmtCharset = stmt.Charset
} else {
// Streamer sends a nil Charset for statements that use the
// charset we specified in the request.
stmtCharset = blp.defaultCharset
}
if *blp.currentCharset != *stmtCharset {
// In regular MySQL replication, the charset is silently adjusted as
// needed during event playback. Here we also adjust so that playback
// proceeds, but in Vitess-land this usually means a misconfigured
// server or a misbehaving client, so we spam the logs with warnings.
log.Warningf("BinlogPlayer changing charset from %v to %v for statement %d in transaction %v", blp.currentCharset, stmtCharset, i, *tx)
err = dbClient.dbConn.SetCharset(stmtCharset)
if err != nil {
return false, fmt.Errorf("can't set charset for statement %d in transaction %v: %v", i, *tx, err)
}
blp.currentCharset = stmtCharset
}
}
if _, err = blp.exec(string(stmt.Sql)); err == nil {
continue
}
if sqlErr, ok := err.(*sqldb.SQLError); ok && sqlErr.Number() == 1213 {
// Deadlock: ask for retry
log.Infof("Deadlock: %v", err)
if err = blp.dbClient.Rollback(); err != nil {
return false, err
}
return false, nil
}
return false, err
}
if err = blp.dbClient.Commit(); err != nil {
return false, fmt.Errorf("failed query COMMIT, err: %s", err)
}
blp.blplStats.Timings.Record(BlplTransaction, txnStartTime)
return true, nil
}
func (blp *BinlogPlayer) exec(sql string) (*sqltypes.Result, error) {
queryStartTime := time.Now()
qr, err := blp.dbClient.ExecuteFetch(sql, 0, false)
blp.blplStats.Timings.Record(BlplQuery, queryStartTime)
if d := time.Now().Sub(queryStartTime); d > SlowQueryThreshold {
log.Infof("SLOW QUERY (took %.2fs) '%s'", d.Seconds(), sql)
}
return qr, err
}
// ApplyBinlogEvents makes an RPC request to BinlogServer
// and processes the events. It will return nil if the provided context
// was canceled, or if we reached the stopping point.
// It will return io.EOF if the server stops sending us updates.
// It may return any other error it encounters.
func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error {
// Instantiate the throttler based on the configuration stored in the db.
maxTPS, maxReplicationLag, err := blp.readThrottlerSettings()
if err != nil {
log.Error(err)
return err
}
t, err := throttler.NewThrottler(
fmt.Sprintf("BinlogPlayer/%d", blp.uid), "transactions", 1 /* threadCount */, maxTPS, maxReplicationLag)
if err != nil {
err := fmt.Errorf("failed to instantiate throttler: %v", err)
log.Error(err)
return err
}
defer t.Close()
// Log the mode of operation and when the player stops.
if len(blp.tables) > 0 {
log.Infof("BinlogPlayer client %v for tables %v starting @ '%v', server: %v",
blp.uid,
blp.tables,
blp.position,
blp.tablet,
)
} else {
log.Infof("BinlogPlayer client %v for keyrange '%v-%v' starting @ '%v', server: %v",
blp.uid,
hex.EncodeToString(blp.keyRange.Start),
hex.EncodeToString(blp.keyRange.End),
blp.position,
blp.tablet,
)
}
if !blp.stopPosition.IsZero() {
// We need to stop at some point. Sanity check the point.
switch {
case blp.position.Equal(blp.stopPosition):
log.Infof("Not starting BinlogPlayer, we're already at the desired position %v", blp.stopPosition)
return nil
case blp.position.AtLeast(blp.stopPosition):
return fmt.Errorf("starting point %v greater than stopping point %v", blp.position, blp.stopPosition)
default:
log.Infof("Will stop player when reaching %v", blp.stopPosition)
}
}
clientFactory, ok := clientFactories[*binlogPlayerProtocol]
if !ok {
return fmt.Errorf("no binlog player client factory named %v", *binlogPlayerProtocol)
}
blplClient := clientFactory()
err = blplClient.Dial(blp.tablet, *BinlogPlayerConnTimeout)
if err != nil {
err := fmt.Errorf("error dialing binlog server: %v", err)
log.Error(err)
return err
}
defer blplClient.Close()
// Get the current charset of our connection, so we can ask the stream server
// to check that they match. The streamer will also only send per-statement
// charset data if that statement's charset is different from what we specify.
if dbClient, ok := blp.dbClient.(*DBClient); ok {
blp.defaultCharset, err = dbClient.dbConn.GetCharset()
if err != nil {
return fmt.Errorf("can't get charset to request binlog stream: %v", err)
}
log.Infof("original charset: %v", blp.defaultCharset)
blp.currentCharset = blp.defaultCharset
// Restore original charset when we're done.
defer func() {
log.Infof("restoring original charset %v", blp.defaultCharset)
if csErr := dbClient.dbConn.SetCharset(blp.defaultCharset); csErr != nil {
log.Errorf("can't restore original charset %v: %v", blp.defaultCharset, csErr)
}
}()
}
var stream BinlogTransactionStream
if len(blp.tables) > 0 {
stream, err = blplClient.StreamTables(ctx, replication.EncodePosition(blp.position), blp.tables, blp.defaultCharset)
} else {
stream, err = blplClient.StreamKeyRange(ctx, replication.EncodePosition(blp.position), blp.keyRange, blp.defaultCharset)
}
if err != nil {
err := fmt.Errorf("error sending streaming query to binlog server: %v", err)
log.Error(err)
return err
}
for {
// Block if we are throttled.
for {
backoff := t.Throttle(0 /* threadID */)
if backoff == throttler.NotThrottled {
break
}
// We don't bother checking for context cancellation here because the
// sleep will block only up to 1 second. (Usually, backoff is 1s / rate
// e.g. a rate of 1000 TPS results into a backoff of 1 ms.)
time.Sleep(backoff)
}
// get the response
response, err := stream.Recv()
if err != nil {
switch err {
case context.Canceled:
return nil
default:
// if the context is canceled, we
// return nil (some RPC
// implementations will remap the
// context error to their own errors)
select {
case <-ctx.Done():
if ctx.Err() == context.Canceled {
return nil
}
default:
}
return fmt.Errorf("Error received from Stream %v", err)
}
}
// process the transaction
for {
ok, err = blp.processTransaction(response)
if err != nil {
return fmt.Errorf("Error in processing binlog event %v", err)
}
if ok {
if !blp.stopPosition.IsZero() {
if blp.position.AtLeast(blp.stopPosition) {
log.Infof("Reached stopping position, done playing logs")
return nil
}
}
break
}
log.Infof("Retrying txn")
time.Sleep(1 * time.Second)
}
}
}
// CreateBlpCheckpoint returns the statements required to create
// the _vt.blp_checkpoint table
func CreateBlpCheckpoint() []string {
return []string{
"CREATE DATABASE IF NOT EXISTS _vt",
`CREATE TABLE IF NOT EXISTS _vt.blp_checkpoint (
source_shard_uid INT(10) UNSIGNED NOT NULL,
pos VARCHAR(250) DEFAULT NULL,
max_tps BIGINT(20) NOT NULL,
max_replication_lag BIGINT(20) NOT NULL,
time_updated BIGINT(20) UNSIGNED NOT NULL,
transaction_timestamp BIGINT(20) UNSIGNED NOT NULL,
flags VARCHAR(250) DEFAULT NULL,
PRIMARY KEY (source_shard_uid)
) ENGINE=InnoDB DEFAULT CHARSET=utf8`}
}
// PopulateBlpCheckpoint returns a statement to populate the first value into
// the _vt.blp_checkpoint table.
func PopulateBlpCheckpoint(index uint32, position string, maxTPS int64, maxReplicationLag int64, timeUpdated int64, flags string) string {
return fmt.Sprintf("INSERT INTO _vt.blp_checkpoint "+
"(source_shard_uid, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, flags) "+
"VALUES (%v, '%v', %v, %v, %v, 0, '%v')",
index, position, maxTPS, maxReplicationLag, timeUpdated, flags)
}
// updateBlpCheckpoint returns a statement to update a value in the
// _vt.blp_checkpoint table.
func updateBlpCheckpoint(uid uint32, pos replication.Position, timeUpdated int64, txTimestamp int64) string {
if txTimestamp != 0 {
return fmt.Sprintf(
"UPDATE _vt.blp_checkpoint "+
"SET pos='%v', time_updated=%v, transaction_timestamp=%v "+
"WHERE source_shard_uid=%v",
replication.EncodePosition(pos), timeUpdated, txTimestamp, uid)
}
return fmt.Sprintf(
"UPDATE _vt.blp_checkpoint "+
"SET pos='%v', time_updated=%v "+
"WHERE source_shard_uid=%v",
replication.EncodePosition(pos), timeUpdated, uid)
}
// QueryBlpCheckpoint returns a statement to query the gtid and flags for a
// given shard from the _vt.blp_checkpoint table.
func QueryBlpCheckpoint(index uint32) string {
return fmt.Sprintf("SELECT pos, flags FROM _vt.blp_checkpoint WHERE source_shard_uid=%v", index)
}
// QueryBlpThrottlerSettings returns a statement to query the throttler settings
// (used by filtered replication) for a given shard from the_vt.blp_checkpoint
// table.
func QueryBlpThrottlerSettings(index uint32) string {
return fmt.Sprintf("SELECT max_tps, max_replication_lag FROM _vt.blp_checkpoint WHERE source_shard_uid=%v", index)
}