github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/relay/upstream_reader.go (about) 1 // Copyright 2019 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package relay 15 16 import ( 17 "context" 18 "sync" 19 20 "github.com/go-mysql-org/go-mysql/mysql" 21 "github.com/go-mysql-org/go-mysql/replication" 22 "github.com/pingcap/tiflow/dm/pkg/binlog/common" 23 "github.com/pingcap/tiflow/dm/pkg/binlog/reader" 24 "github.com/pingcap/tiflow/dm/pkg/log" 25 "github.com/pingcap/tiflow/dm/pkg/terror" 26 "go.uber.org/zap" 27 ) 28 29 // RResult represents a read operation result. 30 type RResult struct { 31 Event *replication.BinlogEvent 32 } 33 34 // Reader reads binlog events from a upstream master server. 35 // The read binlog events should be send to a transformer. 36 // The reader should support: 37 // 1. handle expected errors 38 // 2. do retry if possible 39 // 40 // NOTE: some errors still need to be handled in the outer caller. 41 type Reader interface { 42 // Start starts the reading process. 43 Start() error 44 45 // Close closes the reader and release the resource. 46 Close() error 47 48 // GetEvent gets the binlog event one by one, it will block if no event can be read. 49 // You can pass a context (like Cancel) to break the block. 50 GetEvent(ctx context.Context) (RResult, error) 51 } 52 53 // RConfig is the configuration used by the Reader. 54 type RConfig struct { 55 SyncConfig replication.BinlogSyncerConfig 56 Pos mysql.Position 57 GTIDs mysql.GTIDSet 58 EnableGTID bool 59 MasterID string // the identifier for the master, used when logging. 60 } 61 62 // reader implements Reader interface. 63 type upstreamReader struct { 64 cfg *RConfig 65 66 mu sync.RWMutex 67 stage common.Stage 68 69 in reader.Reader // the underlying reader used to read binlog events. 70 out chan *replication.BinlogEvent 71 72 logger log.Logger 73 } 74 75 // NewUpstreamReader creates a Reader instance. 76 func NewUpstreamReader(cfg *RConfig) Reader { 77 return &upstreamReader{ 78 cfg: cfg, 79 in: reader.NewTCPReader(cfg.SyncConfig), 80 out: make(chan *replication.BinlogEvent), 81 logger: log.With(zap.String("component", "relay reader")), 82 } 83 } 84 85 // Start implements Reader.Start. 86 func (r *upstreamReader) Start() error { 87 r.mu.Lock() 88 defer r.mu.Unlock() 89 90 if r.stage != common.StageNew { 91 return terror.ErrRelayReaderNotStateNew.Generate(r.stage, common.StageNew) 92 } 93 r.stage = common.StagePrepared 94 95 defer func() { 96 status := r.in.Status() 97 r.logger.Info("set up binlog reader", zap.String("master", r.cfg.MasterID), zap.Reflect("status", status)) 98 }() 99 100 var err error 101 if r.cfg.EnableGTID { 102 err = r.setUpReaderByGTID() 103 } else { 104 err = r.setUpReaderByPos() 105 } 106 107 return err 108 } 109 110 // Close implements Reader.Close. 111 func (r *upstreamReader) Close() error { 112 r.mu.Lock() 113 defer r.mu.Unlock() 114 115 if r.stage != common.StagePrepared { 116 return terror.ErrRelayReaderStateCannotClose.Generate(r.stage, common.StagePrepared) 117 } 118 119 err := r.in.Close() 120 r.stage = common.StageClosed 121 return err 122 } 123 124 // GetEvent implements Reader.GetEvent. 125 // NOTE: can only close the reader after this returned. 126 func (r *upstreamReader) GetEvent(ctx context.Context) (RResult, error) { 127 r.mu.RLock() 128 defer r.mu.RUnlock() 129 130 var result RResult 131 if r.stage != common.StagePrepared { 132 return result, terror.ErrRelayReaderNeedStart.Generate(r.stage, common.StagePrepared) 133 } 134 135 ev, err := r.in.GetEvent(ctx) 136 137 if err == nil { 138 result.Event = ev 139 } 140 return result, err 141 } 142 143 func (r *upstreamReader) setUpReaderByGTID() error { 144 gs := r.cfg.GTIDs 145 r.logger.Info("start sync", zap.String("master", r.cfg.MasterID), zap.Stringer("from GTID set", gs)) 146 return r.in.StartSyncByGTID(gs) 147 } 148 149 func (r *upstreamReader) setUpReaderByPos() error { 150 pos := r.cfg.Pos 151 r.logger.Info("start sync", zap.String("master", r.cfg.MasterID), zap.Stringer("from position", pos)) 152 return r.in.StartSyncByPos(pos) 153 }