github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/binlog/reader/tcp.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 reader 15 16 import ( 17 "context" 18 "encoding/json" 19 "sync" 20 21 gmysql "github.com/go-mysql-org/go-mysql/mysql" 22 "github.com/go-mysql-org/go-mysql/replication" 23 "github.com/pingcap/failpoint" 24 "github.com/pingcap/tiflow/dm/pkg/binlog/common" 25 "github.com/pingcap/tiflow/dm/pkg/log" 26 "github.com/pingcap/tiflow/dm/pkg/terror" 27 "go.uber.org/zap" 28 ) 29 30 // TCPReader is a binlog event reader which read binlog events from a TCP stream. 31 type TCPReader struct { 32 syncerCfg replication.BinlogSyncerConfig 33 34 mu sync.RWMutex 35 stage common.Stage 36 37 syncer *replication.BinlogSyncer 38 streamer *replication.BinlogStreamer 39 } 40 41 // TCPReaderStatus represents the status of a TCPReader. 42 type TCPReaderStatus struct { 43 Stage string `json:"stage"` 44 ConnID uint32 `json:"connection"` 45 } 46 47 // String implements Stringer.String. 48 func (s *TCPReaderStatus) String() string { 49 data, err := json.Marshal(s) 50 if err != nil { 51 log.L().Error("fail to marshal status to json", zap.Reflect("tcp reader status", s), log.ShortError(err)) 52 } 53 return string(data) 54 } 55 56 // NewTCPReader creates a TCPReader instance. 57 func NewTCPReader(syncerCfg replication.BinlogSyncerConfig) Reader { 58 return &TCPReader{ 59 syncerCfg: syncerCfg, 60 syncer: replication.NewBinlogSyncer(syncerCfg), 61 } 62 } 63 64 // StartSyncByPos implements Reader.StartSyncByPos. 65 func (r *TCPReader) StartSyncByPos(pos gmysql.Position) error { 66 r.mu.Lock() 67 defer r.mu.Unlock() 68 69 if r.stage != common.StageNew { 70 return terror.ErrRelayReaderNotStateNew.Generate(r.stage, common.StageNew) 71 } 72 73 failpoint.Inject("MockTCPReaderStartSyncByPos", func() { 74 r.stage = common.StagePrepared 75 failpoint.Return(nil) 76 }) 77 78 streamer, err := r.syncer.StartSync(pos) 79 if err != nil { 80 return terror.ErrRelayTCPReaderStartSync.Delegate(err, pos) 81 } 82 83 r.streamer = streamer 84 r.stage = common.StagePrepared 85 return nil 86 } 87 88 // StartSyncByGTID implements Reader.StartSyncByGTID. 89 func (r *TCPReader) StartSyncByGTID(gSet gmysql.GTIDSet) error { 90 r.mu.Lock() 91 defer r.mu.Unlock() 92 93 if r.stage != common.StageNew { 94 return terror.ErrRelayReaderNotStateNew.Generate(r.stage, common.StageNew) 95 } 96 97 if gSet == nil { 98 return terror.ErrRelayTCPReaderNilGTID.Generate() 99 } 100 101 failpoint.Inject("MockTCPReaderStartSyncByGTID", func() { 102 r.stage = common.StagePrepared 103 failpoint.Return(nil) 104 }) 105 106 streamer, err := r.syncer.StartSyncGTID(gSet) 107 if err != nil { 108 return terror.ErrRelayTCPReaderStartSyncGTID.Delegate(err, gSet) 109 } 110 111 r.streamer = streamer 112 r.stage = common.StagePrepared 113 return nil 114 } 115 116 // Close implements Reader.Close. 117 func (r *TCPReader) Close() error { 118 r.mu.Lock() 119 defer r.mu.Unlock() 120 121 if r.stage != common.StagePrepared { 122 return terror.ErrRelayReaderStateCannotClose.Generate(r.stage, common.StagePrepared) 123 } 124 125 failpoint.Inject("MockTCPReaderClose", func() { 126 r.stage = common.StageClosed 127 failpoint.Return(nil) 128 }) 129 130 // unclosed conn bug already fixed in go-mysql, https://github.com/go-mysql-org/go-mysql/pull/411 131 r.syncer.Close() 132 r.stage = common.StageClosed 133 return nil 134 } 135 136 // GetEvent implements Reader.GetEvent. 137 func (r *TCPReader) GetEvent(ctx context.Context) (*replication.BinlogEvent, error) { 138 r.mu.RLock() 139 defer r.mu.RUnlock() 140 141 if r.stage != common.StagePrepared { 142 return nil, terror.ErrRelayReaderNeedStart.Generate(r.stage, common.StagePrepared) 143 } 144 145 failpoint.Inject("MockTCPReaderGetEvent", func() { 146 failpoint.Return(nil, nil) 147 }) 148 149 ev, err := r.streamer.GetEvent(ctx) 150 return ev, terror.ErrRelayTCPReaderGetEvent.Delegate(err) 151 } 152 153 // Status implements Reader.Status. 154 func (r *TCPReader) Status() interface{} { 155 r.mu.RLock() 156 stage := r.stage 157 r.mu.RUnlock() 158 159 failpoint.Inject("MockTCPReaderStatus", func() { 160 status := &TCPReaderStatus{ 161 Stage: stage.String(), 162 ConnID: uint32(1), 163 } 164 failpoint.Return(status) 165 }) 166 var connID uint32 167 if stage != common.StageNew { 168 connID = r.syncer.LastConnectionID() 169 } 170 return &TCPReaderStatus{ 171 Stage: stage.String(), 172 ConnID: connID, 173 } 174 }